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/05 03:12:59 UTC

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

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x f9e915b3d -> 4b6794368


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
new file mode 100644
index 0000000..bc2a14d
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterRanking.java
@@ -0,0 +1,339 @@
+/*
+ * 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.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.TestUtil;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Random;
+
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestUnifiedHighlighterRanking extends LuceneTestCase {
+
+  Analyzer indexAnalyzer;
+
+  // note: don't choose reanalysis because it doesn't always know the term frequency, which is a statistic used
+  //   in passage ranking.  Sometimes it does (e.g. when it builds a MemoryIndex) but not necessarily.
+  final FieldType fieldType = UHTestHelper.randomFieldType(random(), UHTestHelper.postingsType, UHTestHelper.tvType);
+
+  /**
+   * indexes a bunch of gibberish, and then highlights top(n).
+   * asserts that top(n) highlights is a subset of top(n+1) up to some max N
+   */
+  // TODO: this only tests single-valued fields. we should also index multiple values per field!
+  public void testRanking() throws Exception {
+    // number of documents: we will check each one
+    final int numDocs = atLeast(100);
+    // number of top-N snippets, we will check 1 .. N
+    final int maxTopN = 5;
+    // maximum number of elements to put in a sentence.
+    final int maxSentenceLength = 10;
+    // maximum number of sentences in a document
+    final int maxNumSentences = 20;
+
+    Directory dir = newDirectory();
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+    Document document = new Document();
+    Field id = new StringField("id", "", Field.Store.NO);
+    Field body = new Field("body", "", fieldType);
+    document.add(id);
+    document.add(body);
+
+    for (int i = 0; i < numDocs; i++) {
+      StringBuilder bodyText = new StringBuilder();
+      int numSentences = TestUtil.nextInt(random(), 1, maxNumSentences);
+      for (int j = 0; j < numSentences; j++) {
+        bodyText.append(newSentence(random(), maxSentenceLength));
+      }
+      body.setStringValue(bodyText.toString());
+      id.setStringValue(Integer.toString(i));
+      iw.addDocument(document);
+    }
+
+    IndexReader ir = iw.getReader();
+    IndexSearcher searcher = newSearcher(ir);
+    for (int i = 0; i < numDocs; i++) {
+      checkDocument(searcher, i, maxTopN);
+    }
+    iw.close();
+    ir.close();
+    dir.close();
+  }
+
+  private void checkDocument(IndexSearcher is, int doc, int maxTopN) throws IOException {
+    for (int ch = 'a'; ch <= 'z'; ch++) {
+      Term term = new Term("body", "" + (char) ch);
+      // check a simple term query
+      checkQuery(is, new TermQuery(term), doc, maxTopN);
+      // check a boolean query
+      Term nextTerm = new Term("body", "" + (char) (ch + 1));
+      BooleanQuery bq = new BooleanQuery.Builder()
+          .add(new TermQuery(term), BooleanClause.Occur.SHOULD)
+          .add(new TermQuery(nextTerm), BooleanClause.Occur.SHOULD)
+          .build();
+      checkQuery(is, bq, doc, maxTopN);
+    }
+  }
+
+  private void checkQuery(IndexSearcher is, Query query, int doc, int maxTopN) throws IOException {
+    for (int n = 1; n < maxTopN; n++) {
+      final FakePassageFormatter f1 = new FakePassageFormatter();
+      UnifiedHighlighter p1 = new UnifiedHighlighter(is, indexAnalyzer) {
+        @Override
+        protected PassageFormatter getFormatter(String field) {
+          assertEquals("body", field);
+          return f1;
+        }
+      };
+      p1.setMaxLength(Integer.MAX_VALUE - 1);
+
+      final FakePassageFormatter f2 = new FakePassageFormatter();
+      UnifiedHighlighter p2 = new UnifiedHighlighter(is, indexAnalyzer) {
+        @Override
+        protected PassageFormatter getFormatter(String field) {
+          assertEquals("body", field);
+          return f2;
+        }
+      };
+      p2.setMaxLength(Integer.MAX_VALUE - 1);
+
+      BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
+      queryBuilder.add(query, BooleanClause.Occur.MUST);
+      queryBuilder.add(new TermQuery(new Term("id", Integer.toString(doc))), BooleanClause.Occur.MUST);
+      BooleanQuery bq = queryBuilder.build();
+      TopDocs td = is.search(bq, 1);
+      p1.highlight("body", bq, td, n);
+      p2.highlight("body", bq, td, n + 1);
+      assertTrue(f2.seen.containsAll(f1.seen));
+    }
+  }
+
+  /**
+   * returns a new random sentence, up to maxSentenceLength "words" in length.
+   * each word is a single character (a-z). The first one is capitalized.
+   */
+  private String newSentence(Random r, int maxSentenceLength) {
+    StringBuilder sb = new StringBuilder();
+    int numElements = TestUtil.nextInt(r, 1, maxSentenceLength);
+    for (int i = 0; i < numElements; i++) {
+      if (sb.length() > 0) {
+        sb.append(' ');
+        sb.append((char) TestUtil.nextInt(r, 'a', 'z'));
+      } else {
+        // capitalize the first word to help breakiterator
+        sb.append((char) TestUtil.nextInt(r, 'A', 'Z'));
+      }
+    }
+    sb.append(". "); // finalize sentence
+    return sb.toString();
+  }
+
+  /**
+   * a fake formatter that doesn't actually format passages.
+   * instead it just collects them for asserts!
+   */
+  static class FakePassageFormatter extends PassageFormatter {
+    HashSet<Pair> seen = new HashSet<>();
+
+    @Override
+    public String format(Passage passages[], String content) {
+      for (Passage p : passages) {
+        // verify some basics about the passage
+        assertTrue(p.getScore() >= 0);
+        assertTrue(p.getNumMatches() > 0);
+        assertTrue(p.getStartOffset() >= 0);
+        assertTrue(p.getStartOffset() <= content.length());
+        assertTrue(p.getEndOffset() >= p.getStartOffset());
+        assertTrue(p.getEndOffset() <= content.length());
+        // we use a very simple analyzer. so we can assert the matches are correct
+        int lastMatchStart = -1;
+        for (int i = 0; i < p.getNumMatches(); i++) {
+          BytesRef term = p.getMatchTerms()[i];
+          int matchStart = p.getMatchStarts()[i];
+          assertTrue(matchStart >= 0);
+          // must at least start within the passage
+          assertTrue(matchStart < p.getEndOffset());
+          int matchEnd = p.getMatchEnds()[i];
+          assertTrue(matchEnd >= 0);
+          // always moving forward
+          assertTrue(matchStart >= lastMatchStart);
+          lastMatchStart = matchStart;
+          // single character terms
+          assertEquals(matchStart + 1, matchEnd);
+          // and the offsets must be correct...
+          assertEquals(1, term.length);
+          assertEquals((char) term.bytes[term.offset], Character.toLowerCase(content.charAt(matchStart)));
+        }
+        // record just the start/end offset for simplicity
+        seen.add(new Pair(p.getStartOffset(), p.getEndOffset()));
+      }
+      return "bogus!!!!!!";
+    }
+  }
+
+  static class Pair {
+    final int start;
+    final int end;
+
+    Pair(int start, int end) {
+      this.start = start;
+      this.end = end;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + end;
+      result = prime * result + start;
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      Pair other = (Pair) obj;
+      if (end != other.end) {
+        return false;
+      }
+      if (start != other.start) {
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      return "Pair [start=" + start + ", end=" + end + "]";
+    }
+  }
+
+  /**
+   * sets b=0 to disable passage length normalization
+   */
+  public void testCustomB() throws Exception {
+    Directory dir = newDirectory();
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(indexAnalyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.  This test is a better test but the sentence is excruiatingly long, " +
+        "you have no idea how painful it was for me to type this long sentence into my IDE.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected PassageScorer getScorer(String field) {
+        return new PassageScorer(1.2f, 0, 87);
+      }
+    };
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 1);
+    assertEquals(1, snippets.length);
+    assertTrue(snippets[0].startsWith("This <b>test</b> is a better <b>test</b>"));
+
+    ir.close();
+    dir.close();
+  }
+
+  /**
+   * sets k1=0 for simple coordinate-level match (# of query terms present)
+   */
+  public void testCustomK1() throws Exception {
+    Directory dir = newDirectory();
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(indexAnalyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This has only foo foo. " +
+        "On the other hand this sentence contains both foo and bar. " +
+        "This has only bar bar bar bar bar bar bar bar bar bar bar bar.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected PassageScorer getScorer(String field) {
+        return new PassageScorer(0, 0.75f, 87);
+      }
+    };
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "foo")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("body", "bar")), BooleanClause.Occur.SHOULD)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 1);
+    assertEquals(1, snippets.length);
+    assertTrue(snippets[0].startsWith("On the other hand"));
+
+    ir.close();
+    dir.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterReanalysis.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterReanalysis.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterReanalysis.java
new file mode 100644
index 0000000..c881269
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterReanalysis.java
@@ -0,0 +1,74 @@
+/*
+ * 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.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestUnifiedHighlighterReanalysis extends LuceneTestCase {
+
+  private MockAnalyzer indexAnalyzer =
+      new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase;
+
+  @Test
+  public void testWithoutIndexSearcher() throws IOException {
+    String text = "This is a test. Just a test highlighting without a searcher. Feel free to ignore.";
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("title", "test")), BooleanClause.Occur.SHOULD)
+        .build();
+
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(null, indexAnalyzer);
+    String snippet = highlighter.highlightWithoutSearcher("body", query, text, 1).toString();
+
+    assertEquals("Just a test <b>highlighting</b> without a searcher. ", snippet);
+
+    assertEquals("test single space", " ", highlighter.highlightWithoutSearcher("body", query, " ", 1));
+
+    assertEquals("Hello", highlighter.highlightWithoutSearcher("nonexistent", query, "Hello", 1));
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testIndexSearcherNullness() throws IOException {
+    String text = "This is a test. Just a test highlighting without a searcher. Feel free to ignore.";
+    Query query = new TermQuery(new Term("body", "highlighting"));
+
+    try (Directory directory = newDirectory();
+         RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
+         IndexReader indexReader = indexWriter.getReader()) {
+      IndexSearcher searcher = newSearcher(indexReader);
+      UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+      highlighter.highlightWithoutSearcher("body", query, text, 1);//should throw
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java
new file mode 100644
index 0000000..5fecdc6
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterStrictPhrases.java
@@ -0,0 +1,404 @@
+/*
+ * 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 com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchNoDocsQuery;
+import org.apache.lucene.search.MultiPhraseQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.QueryBuilder;
+import org.junit.After;
+import org.junit.Before;
+
+@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestUnifiedHighlighterStrictPhrases extends LuceneTestCase {
+
+  final FieldType fieldType;
+
+  Directory dir;
+  MockAnalyzer indexAnalyzer;
+  RandomIndexWriter indexWriter;
+  IndexSearcher searcher;
+  UnifiedHighlighter highlighter;
+  IndexReader indexReader;
+
+  @ParametersFactory
+  public static Iterable<Object[]> parameters() {
+    return UHTestHelper.parametersFactoryList();
+  }
+
+  public TestUnifiedHighlighterStrictPhrases(FieldType fieldType) {
+    this.fieldType = fieldType;
+  }
+
+  @Before
+  public void doBefore() throws IOException {
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
+    indexAnalyzer.setPositionIncrementGap(3);// more than default
+    dir = newDirectory();
+    indexWriter = new RandomIndexWriter(random(), dir, indexAnalyzer);
+  }
+
+  @After
+  public void doAfter() throws IOException {
+    IOUtils.close(indexReader, indexWriter, dir);
+  }
+
+  private Document newDoc(String... bodyVals) {
+    Document doc = new Document();
+    for (String bodyVal : bodyVals) {
+      doc.add(new Field("body", bodyVal, fieldType));
+    }
+    return doc;
+  }
+
+  private void initReaderSearcherHighlighter() throws IOException {
+    indexReader = indexWriter.getReader();
+    searcher = newSearcher(indexReader);
+    highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setHighlightPhrasesStrictly(true);
+  }
+
+  private PhraseQuery newPhraseQuery(String field, String phrase) {
+    return (PhraseQuery) new QueryBuilder(indexAnalyzer).createPhraseQuery(field, phrase);
+  }
+
+  private PhraseQuery setSlop(PhraseQuery query, int slop) {
+    PhraseQuery.Builder builder = new PhraseQuery.Builder();
+    Term[] terms = query.getTerms();
+    int[] positions = query.getPositions();
+    for (int i = 0; i < terms.length; i++) {
+      builder.add(terms[i], positions[i]);
+    }
+    builder.setSlop(slop);
+    return builder.build();
+  }
+
+  public void testBasics() throws IOException {
+    indexWriter.addDocument(newDoc("Yin yang, filter")); // filter out. test getTermToSpanLists reader 1-doc filter
+    indexWriter.addDocument(newDoc("yin alone, Yin yang, yin gap yang"));
+    initReaderSearcherHighlighter();
+
+    //query:  -filter +"yin yang"
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "filter")), BooleanClause.Occur.MUST_NOT)
+        .add(newPhraseQuery("body", "yin yang"), BooleanClause.Occur.MUST)
+        .build();
+
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"yin alone, <b>Yin</b> <b>yang</b>, yin gap yang"}, snippets);
+  }
+
+  public void testWithSameTermQuery() throws IOException {
+    indexWriter.addDocument(newDoc("Yin yang, yin gap yang"));
+    initReaderSearcherHighlighter();
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "yin")), BooleanClause.Occur.MUST)
+        .add(newPhraseQuery("body", "yin yang"), BooleanClause.Occur.MUST)
+        // add queries for other fields; we shouldn't highlight these because of that.
+        .add(new TermQuery(new Term("title", "yang")), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>Yin</b> <b>yang</b>, <b>yin</b> gap yang"}, snippets);
+  }
+
+  public void testPhraseNotInDoc() throws IOException {
+    indexWriter.addDocument(newDoc("Whatever yin")); // query matches this; highlight it
+    indexWriter.addDocument(newDoc("nextdoc yin"));// query does NOT match this, only the SHOULD clause does
+    initReaderSearcherHighlighter();
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        //MUST:
+        .add(new TermQuery(new Term("body", "whatever")), BooleanClause.Occur.MUST)
+        //SHOULD: (yet won't)
+        .add(newPhraseQuery("body", "nextdoc yin"), BooleanClause.Occur.SHOULD)
+        .add(newPhraseQuery("body", "nonexistent yin"), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>Whatever</b> yin"}, snippets);
+  }
+
+  public void testSubPhrases() throws IOException {
+    indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
+    initReaderSearcherHighlighter();
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(newPhraseQuery("body", "alpha bravo charlie"), BooleanClause.Occur.MUST)
+        .add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"}, snippets);
+  }
+
+  public void testSynonyms() throws IOException {
+    indexWriter.addDocument(newDoc("mother father w mom father w dad"));
+    initReaderSearcherHighlighter();
+
+    MultiPhraseQuery query = new MultiPhraseQuery.Builder()
+        .add(new Term[]{new Term("body", "mom"), new Term("body", "mother")})
+        .add(new Term[]{new Term("body", "dad"), new Term("body", "father")})
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>mother</b> <b>father</b> w <b>mom</b> <b>father</b> w dad"}, snippets);
+  }
+
+  /**
+   * Test it does *not* highlight the same term's not next to the span-near.  "charlie" in this case.
+   * This particular example exercises "Rewrite" plus "MTQ" in the same query.
+   */
+  public void testRewriteAndMtq() throws IOException {
+    indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
+    initReaderSearcherHighlighter();
+
+    SpanNearQuery snq = new SpanNearQuery(
+        new SpanQuery[]{
+            new SpanTermQuery(new Term("body", "bravo")),
+            new SpanMultiTermQueryWrapper<>(new PrefixQuery(new Term("body", "ch")))}, // REWRITES
+        0, true);
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(snq, BooleanClause.Occur.MUST)
+        .add(new PrefixQuery(new Term("body", "al")), BooleanClause.Occur.MUST) // MTQ
+        .add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
+        // add queries for other fields; we shouldn't highlight these because of that.
+        .add(newPhraseQuery("title", "bravo alpha"), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo <b>alpha</b>"},
+        snippets);
+
+    // do again, this time with MTQ disabled.  We should only find "alpha bravo".
+    highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
+
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> charlie - charlie bravo alpha"},
+        snippets);
+  }
+
+  /**
+   * Like {@link #testRewriteAndMtq} but no freestanding MTQ
+   */
+  public void testRewrite() throws IOException {
+    indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
+    initReaderSearcherHighlighter();
+
+    SpanNearQuery snq = new SpanNearQuery(
+        new SpanQuery[]{
+            new SpanTermQuery(new Term("body", "bravo")),
+            new SpanMultiTermQueryWrapper<>(new PrefixQuery(new Term("body", "ch")))}, // REWRITES
+        0, true);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(snq, BooleanClause.Occur.MUST)
+//          .add(new PrefixQuery(new Term("body", "al")), BooleanClause.Occur.MUST) // MTQ
+        .add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
+        // add queries for other fields; we shouldn't highlight these because of that.
+        .add(newPhraseQuery("title", "bravo alpha"), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"},
+        snippets);
+
+    // do again, this time with MTQ disabled.  We should only find "alpha bravo".
+    highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
+
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> charlie - charlie bravo alpha"},
+        snippets);
+  }
+
+  /**
+   * Like {@link #testRewriteAndMtq} but no rewrite.
+   */
+  public void testMtq() throws IOException {
+    indexWriter.addDocument(newDoc("alpha bravo charlie - charlie bravo alpha"));
+    initReaderSearcherHighlighter();
+
+    SpanNearQuery snq = new SpanNearQuery(
+        new SpanQuery[]{
+            new SpanTermQuery(new Term("body", "bravo")),
+            new SpanTermQuery(new Term("body", "charlie"))}, // does NOT rewrite
+        0, true);
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(snq, BooleanClause.Occur.MUST)
+        .add(new PrefixQuery(new Term("body", "al")), BooleanClause.Occur.MUST) // MTQ
+        .add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
+        // add queries for other fields; we shouldn't highlight these because of that.
+        .add(newPhraseQuery("title", "bravo alpha"), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo <b>alpha</b>"},
+        snippets);
+
+    // do again, this time with MTQ disabled.
+    highlighter.setHandleMultiTermQuery(false);//disable but leave phrase processing enabled
+
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    snippets = highlighter.highlight("body", query, topDocs);
+
+    assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> - charlie bravo alpha"},
+        snippets);
+  }
+
+  public void testMultiValued() throws IOException {
+    indexWriter.addDocument(newDoc("one bravo three", "four bravo six"));
+    initReaderSearcherHighlighter();
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(newPhraseQuery("body", "one bravo"), BooleanClause.Occur.MUST)
+        .add(newPhraseQuery("body", "four bravo"), BooleanClause.Occur.MUST)
+        .add(new PrefixQuery(new Term("body", "br")), BooleanClause.Occur.MUST)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs, 2);
+
+    assertArrayEquals(new String[]{"<b>one</b> <b>bravo</b> three... <b>four</b> <b>bravo</b> six"},
+        snippets);
+
+
+    // now test phraseQuery won't span across values
+    assert indexAnalyzer.getPositionIncrementGap("body") > 0;
+
+    PhraseQuery phraseQuery = newPhraseQuery("body", "three four");
+    // 1 too little; won't span
+    phraseQuery = setSlop(phraseQuery, indexAnalyzer.getPositionIncrementGap("body") - 1);
+
+    query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "bravo")), BooleanClause.Occur.MUST)
+        .add(phraseQuery, BooleanClause.Occur.SHOULD)
+        .build();
+
+    topDocs = searcher.search(query, 10);
+    snippets = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals("one <b>bravo</b> three... four <b>bravo</b> six", snippets[0]);
+
+    // and add just enough slop to cross the values:
+    phraseQuery = newPhraseQuery("body", "three four");
+    phraseQuery = setSlop(phraseQuery, indexAnalyzer.getPositionIncrementGap("body")); // just enough to span
+    query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "bravo")), BooleanClause.Occur.MUST)
+        .add(phraseQuery, BooleanClause.Occur.MUST) // must match and it will
+        .build();
+    topDocs = searcher.search(query, 10);
+    assertEquals(1, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals("one <b>bravo</b> <b>three</b>... <b>four</b> <b>bravo</b> six", snippets[0]);
+  }
+
+  public void testMaxLen() throws IOException {
+    indexWriter.addDocument(newDoc("alpha bravo charlie - gap alpha bravo")); // hyphen is at char 21
+    initReaderSearcherHighlighter();
+    highlighter.setMaxLength(21);
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(newPhraseQuery("body", "alpha bravo"), BooleanClause.Occur.MUST)
+        .add(newPhraseQuery("body", "gap alpha"), BooleanClause.Occur.MUST)
+        .add(newPhraseQuery("body", "charlie gap"), BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+
+    if (fieldType == UHTestHelper.reanalysisType) {
+      assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> charlie -"}, snippets);
+    } else {
+      assertArrayEquals(new String[]{"<b>alpha</b> <b>bravo</b> <b>charlie</b> -"}, snippets);
+    }
+  }
+
+  public void testFilteredOutSpan() throws IOException {
+    indexWriter.addDocument(newDoc("freezing cold stuff like stuff freedom of speech"));
+    initReaderSearcherHighlighter();
+
+    WildcardQuery wildcardQuery = new WildcardQuery(new Term("body", "free*"));
+    SpanMultiTermQueryWrapper<WildcardQuery> wildcardSpanQuery = new SpanMultiTermQueryWrapper<>(wildcardQuery);
+    SpanTermQuery termQuery = new SpanTermQuery(new Term("body", "speech"));
+    SpanQuery spanQuery = new SpanNearQuery(new SpanQuery[]{wildcardSpanQuery, termQuery}, 3, false);
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(spanQuery, BooleanClause.Occur.MUST)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String[] snippets = highlighter.highlight("body", query, topDocs);
+    assertArrayEquals(new String[]{"freezing cold stuff like stuff <b>freedom</b> of <b>speech</b>"}, snippets);
+  }
+
+  public void testMatchNoDocsQuery() throws IOException {
+    highlighter = new UnifiedHighlighter(null, indexAnalyzer);
+    highlighter.setHighlightPhrasesStrictly(true);
+    String content = "whatever";
+    Object o = highlighter.highlightWithoutSearcher("body", new MatchNoDocsQuery(), content, 1);
+    assertEquals(content, o);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterTermVec.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterTermVec.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterTermVec.java
new file mode 100644
index 0000000..57d398b
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterTermVec.java
@@ -0,0 +1,182 @@
+/*
+ * 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.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests highlighting for matters *expressly* relating to term vectors.
+ * <p>
+ * This test DOES NOT represent all testing for highlighting when term vectors are used.  Other tests pick the offset
+ * source at random (to include term vectors) and in-effect test term vectors generally.
+ */
+@LuceneTestCase.SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestUnifiedHighlighterTermVec extends LuceneTestCase {
+
+  private Analyzer indexAnalyzer;
+  private Directory dir;
+
+  @Before
+  public void doBefore() throws IOException {
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
+    dir = newDirectory();
+  }
+
+  @After
+  public void doAfter() throws IOException {
+    dir.close();
+  }
+
+  public void testFetchTermVecsOncePerDoc() throws IOException {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    // Declare some number of fields with random field type; but at least one will have term vectors.
+    final int numTvFields = 1 + random().nextInt(3);
+    List<String> fields = new ArrayList<>(numTvFields);
+    List<FieldType> fieldTypes = new ArrayList<>(numTvFields);
+    for (int i = 0; i < numTvFields; i++) {
+      fields.add("body" + i);
+      fieldTypes.add(UHTestHelper.randomFieldType(random()));
+    }
+    //ensure at least one has TVs by setting one randomly to it:
+    fieldTypes.set(random().nextInt(fieldTypes.size()), UHTestHelper.tvType);
+
+    final int numDocs = 1 + random().nextInt(3);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      for (String field : fields) {
+        doc.add(new Field(field, "some test text", UHTestHelper.tvType));
+      }
+      iw.addDocument(doc);
+    }
+
+    // Wrap the reader to ensure we only fetch TVs once per doc
+    DirectoryReader originalReader = iw.getReader();
+    IndexReader ir = new AssertOnceTermVecDirectoryReader(originalReader);
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
+    for (String field : fields) {
+      queryBuilder.add(new TermQuery(new Term(field, "test")), BooleanClause.Occur.MUST);
+    }
+    BooleanQuery query = queryBuilder.build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(numDocs, topDocs.totalHits);
+    Map<String, String[]> fieldToSnippets =
+        highlighter.highlightFields(fields.toArray(new String[numTvFields]), query, topDocs);
+    String[] expectedSnippetsByDoc = new String[numDocs];
+    Arrays.fill(expectedSnippetsByDoc, "some <b>test</b> text");
+    for (String field : fields) {
+      assertArrayEquals(expectedSnippetsByDoc, fieldToSnippets.get(field));
+    }
+
+    ir.close();
+  }
+
+  private static class AssertOnceTermVecDirectoryReader extends FilterDirectoryReader {
+    static final SubReaderWrapper SUB_READER_WRAPPER = new SubReaderWrapper() {
+      @Override
+      public LeafReader wrap(LeafReader reader) {
+        return new FilterLeafReader(reader) {
+          BitSet seenDocIDs = new BitSet();
+
+          @Override
+          public Fields getTermVectors(int docID) throws IOException {
+            // if we're invoked by ParallelLeafReader then we can't do our assertion. TODO see LUCENE-6868
+            if (calledBy(ParallelLeafReader.class) == false
+                && calledBy(CheckIndex.class) == false) {
+              assertFalse("Should not request TVs for doc more than once.", seenDocIDs.get(docID));
+              seenDocIDs.set(docID);
+            }
+
+            return super.getTermVectors(docID);
+          }
+        };
+      }
+    };
+
+    AssertOnceTermVecDirectoryReader(DirectoryReader in) throws IOException {
+      super(in, SUB_READER_WRAPPER);
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
+      return new AssertOnceTermVecDirectoryReader(in);
+    }
+  }
+
+  private static boolean calledBy(Class<?> clazz) {
+    for (StackTraceElement stackTraceElement : Thread.currentThread().getStackTrace()) {
+      if (stackTraceElement.getClassName().equals(clazz.getName()))
+        return true;
+    }
+    return false;
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testUserFailedToIndexOffsets() throws IOException {
+    FieldType fieldType = new FieldType(UHTestHelper.tvType); // note: it's indexed too
+    fieldType.setStoreTermVectorPositions(random().nextBoolean());
+    fieldType.setStoreTermVectorOffsets(false);
+
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+    Document doc = new Document();
+    doc.add(new Field("body", "term vectors", fieldType));
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    TermQuery query = new TermQuery(new Term("body", "vectors"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    try {
+      highlighter.highlight("body", query, topDocs, 1);//should throw
+    } finally {
+      ir.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/UHTestHelper.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/UHTestHelper.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/UHTestHelper.java
new file mode 100644
index 0000000..67400be
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/UHTestHelper.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search.uhighlight;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexOptions;
+
+/**
+ * Helper for {@link UnifiedHighlighter} tests.
+ */
+class UHTestHelper {
+
+  static final FieldType postingsType = new FieldType(TextField.TYPE_STORED);
+  static final FieldType tvType = new FieldType(TextField.TYPE_STORED);
+  static final FieldType postingsWithTvType = new FieldType(TextField.TYPE_STORED);
+  static final FieldType reanalysisType = TextField.TYPE_STORED;
+
+  static {
+    postingsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    postingsType.freeze();
+
+    tvType.setStoreTermVectors(true);
+    tvType.setStoreTermVectorPositions(true);
+    tvType.setStoreTermVectorOffsets(true);
+    tvType.freeze();
+
+    postingsWithTvType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    postingsWithTvType.setStoreTermVectors(true);
+    postingsWithTvType.freeze();
+
+
+    //re-analysis type needs no further changes.
+  }
+
+  public static FieldType randomFieldType(Random random, FieldType... typePossibilities) {
+    if (typePossibilities == null || typePossibilities.length == 0) {
+      typePossibilities = new FieldType[]{postingsType, tvType, postingsWithTvType, reanalysisType};
+    }
+    return typePossibilities[random.nextInt(typePossibilities.length)];
+  }
+
+  /**
+   * for {@link com.carrotsearch.randomizedtesting.annotations.ParametersFactory}
+   */
+  // https://github.com/carrotsearch/randomizedtesting/blob/master/examples/maven/src/main/java/com/carrotsearch/examples/randomizedrunner/Test007ParameterizedTests.java
+  static Iterable<Object[]> parametersFactoryList() {
+    return Arrays.asList(new Object[][]{
+        {postingsType}, {tvType}, {postingsWithTvType}, {reanalysisType}
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
new file mode 100644
index 0000000..641a835
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java
@@ -0,0 +1,182 @@
+/*
+ * 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.visibility;
+
+import java.io.IOException;
+import java.text.BreakIterator;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.uhighlight.FieldHighlighter;
+import org.apache.lucene.search.uhighlight.FieldOffsetStrategy;
+import org.apache.lucene.search.uhighlight.OffsetsEnum;
+import org.apache.lucene.search.uhighlight.Passage;
+import org.apache.lucene.search.uhighlight.PassageFormatter;
+import org.apache.lucene.search.uhighlight.PassageScorer;
+import org.apache.lucene.search.uhighlight.PhraseHelper;
+import org.apache.lucene.search.uhighlight.UnifiedHighlighter;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.junit.Test;
+
+public class TestUnifiedHighlighterExtensibility extends LuceneTestCase {
+
+  /**
+   * This test is for maintaining the extensibility of the FieldOffsetStrategy
+   * for customizations out of package.
+   */
+  @Test
+  public void testFieldOffsetStrategyExtensibility() {
+    final UnifiedHighlighter.OffsetSource offsetSource = UnifiedHighlighter.OffsetSource.NONE_NEEDED;
+    FieldOffsetStrategy strategy = new FieldOffsetStrategy("field",
+        new BytesRef[0],
+        PhraseHelper.NONE,
+        new CharacterRunAutomaton[0]) {
+      @Override
+      public UnifiedHighlighter.OffsetSource getOffsetSource() {
+        return offsetSource;
+      }
+
+      @Override
+      public List<OffsetsEnum> getOffsetsEnums(IndexReader reader, int docId, String content) throws IOException {
+        return Collections.emptyList();
+      }
+
+    };
+    assertEquals(offsetSource, strategy.getOffsetSource());
+  }
+
+  /**
+   * This test is for maintaining the extensibility of the UnifiedHighlighter
+   * for customizations out of package.
+   */
+  @Test
+  public void testUnifiedHighlighterExtensibility() {
+    final int maxLength = 1000;
+    UnifiedHighlighter uh = new UnifiedHighlighter(null, new MockAnalyzer(random())){
+
+      @Override
+      protected Map<String, Object[]> highlightFieldsAsObjects(String[] fieldsIn, Query query, int[] docIdsIn, int[] maxPassagesIn) throws IOException {
+        return super.highlightFieldsAsObjects(fieldsIn, query, docIdsIn, maxPassagesIn);
+      }
+
+      @Override
+      protected OffsetSource getOffsetSource(String field) {
+        return super.getOffsetSource(field);
+      }
+
+      @Override
+      protected BreakIterator getBreakIterator(String field) {
+        return super.getBreakIterator(field);
+      }
+
+      @Override
+      protected PassageScorer getScorer(String field) {
+        return super.getScorer(field);
+      }
+
+      @Override
+      protected PassageFormatter getFormatter(String field) {
+        return super.getFormatter(field);
+      }
+
+      @Override
+      public Analyzer getIndexAnalyzer() {
+        return super.getIndexAnalyzer();
+      }
+
+      @Override
+      public IndexSearcher getIndexSearcher() {
+        return super.getIndexSearcher();
+      }
+
+      @Override
+      protected int getMaxNoHighlightPassages(String field) {
+        return super.getMaxNoHighlightPassages(field);
+      }
+
+      @Override
+      protected Boolean requiresRewrite(SpanQuery spanQuery) {
+        return super.requiresRewrite(spanQuery);
+      }
+
+      @Override
+      protected LimitedStoredFieldVisitor newLimitedStoredFieldsVisitor(String[] fields) {
+        return super.newLimitedStoredFieldsVisitor(fields);
+      }
+
+      @Override
+      protected List<CharSequence[]> loadFieldValues(String[] fields, DocIdSetIterator docIter, int cacheCharsThreshold) throws IOException {
+        return super.loadFieldValues(fields, docIter, cacheCharsThreshold);
+      }
+
+      @Override
+      protected FieldHighlighter getFieldHighlighter(String field, Query query, SortedSet<Term> allTerms, int maxPassages) {
+        return super.getFieldHighlighter(field, query, allTerms, maxPassages);
+      }
+
+      @Override
+      protected FieldOffsetStrategy getOffsetStrategy(String field, Query query, SortedSet<Term> allTerms) {
+        return super.getOffsetStrategy(field, query, allTerms);
+      }
+
+      @Override
+      public int getMaxLength() {
+        return maxLength;
+      }
+    };
+    assertEquals(uh.getMaxLength(), maxLength);
+  }
+
+  @Test
+  public void testPassageFormatterExtensibility() {
+    final Object formattedResponse = new Object();
+    PassageFormatter formatter = new PassageFormatter() {
+      @Override
+      public Object format(Passage[] passages, String content) {
+        return formattedResponse;
+      }
+    };
+    assertEquals(formattedResponse, formatter.format(new Passage[0], ""));
+  }
+
+  @Test
+  public void testFieldHiglighterExtensibility() {
+    final String fieldName = "fieldName";
+    FieldHighlighter fieldHighlighter = new FieldHighlighter(fieldName, null, null, null, 1, 1, null) {
+      @Override
+      protected Passage[] highlightOffsetsEnums(List<OffsetsEnum> offsetsEnums) throws IOException {
+        return super.highlightOffsetsEnums(offsetsEnums);
+      }
+    };
+
+    assertEquals(fieldHighlighter.getField(), fieldName);
+  }
+
+}


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

Posted by ds...@apache.org.
LUCENE-7438: New UnifiedHighlighter

(cherry picked from commit 722e827)


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

Branch: refs/heads/branch_6x
Commit: 4b6794368df373df1f68ccf27f7556914efeb95e
Parents: f9e915b
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 23:12:47 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/4b679436/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index cedcd31..4ebd619 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -9,6 +9,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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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);
+  }
+}


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

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/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..a09ccf3
--- /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)
+            .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).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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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/4b679436/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;
+    }
+  }
+}


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

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
new file mode 100644
index 0000000..72be180
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/UnifiedHighlighter.java
@@ -0,0 +1,1021 @@
+/*
+ * 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.nio.charset.StandardCharsets;
+import java.text.BreakIterator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.function.Supplier;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.BaseCompositeReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FilterLeafReader;
+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.MultiFields;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.InPlaceMergeSorter;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+
+/**
+ * A Highlighter that can get offsets from either
+ * postings ({@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}),
+ * term vectors ({@link FieldType#setStoreTermVectorOffsets(boolean)}),
+ * or via re-analyzing text.
+ * <p>
+ * This highlighter treats the single original document as the whole corpus, and then scores individual
+ * passages as if they were documents in this corpus. It uses a {@link BreakIterator} to find
+ * passages in the text; by default it breaks using {@link BreakIterator#getSentenceInstance(Locale)
+ * getSentenceInstance(Locale.ROOT)}. It then iterates in parallel (merge sorting by offset) through
+ * the positions of all terms from the query, coalescing those hits that occur in a single passage
+ * into a {@link Passage}, and then scores each Passage using a separate {@link PassageScorer}.
+ * Passages are finally formatted into highlighted snippets with a {@link PassageFormatter}.
+ * <p>
+ * You can customize the behavior by calling some of the setters, or by subclassing and overriding some methods.
+ * Some important hooks:
+ * <ul>
+ * <li>{@link #getBreakIterator(String)}: Customize how the text is divided into passages.
+ * <li>{@link #getScorer(String)}: Customize how passages are ranked.
+ * <li>{@link #getFormatter(String)}: Customize how snippets are formatted.
+ * </ul>
+ * <p>
+ * This is thread-safe.
+ *
+ * @lucene.experimental
+ */
+public class UnifiedHighlighter {
+
+  protected static final char MULTIVAL_SEP_CHAR = (char) 0;
+
+  public static final int DEFAULT_MAX_LENGTH = 10000;
+
+  public static final int DEFAULT_CACHE_CHARS_THRESHOLD = 524288; // ~ 1 MB (2 byte chars)
+
+  static final IndexSearcher EMPTY_INDEXSEARCHER;
+
+  static {
+    try {
+      IndexReader emptyReader = new MultiReader();
+      EMPTY_INDEXSEARCHER = new IndexSearcher(emptyReader);
+      EMPTY_INDEXSEARCHER.setQueryCache(null);
+    } catch (IOException bogus) {
+      throw new RuntimeException(bogus);
+    }
+  }
+
+  protected static final CharacterRunAutomaton[] ZERO_LEN_AUTOMATA_ARRAY = new CharacterRunAutomaton[0];
+
+  protected final IndexSearcher searcher; // if null, can only use highlightWithoutSearcher
+
+  protected final Analyzer indexAnalyzer;
+
+  private boolean defaultHandleMtq = true; // e.g. wildcards
+
+  private boolean defaultHighlightPhrasesStrictly = true; // AKA "accuracy" or "query debugging"
+
+  // private boolean defaultRequireFieldMatch = true; TODO
+
+  private int maxLength = DEFAULT_MAX_LENGTH;
+
+  // BreakIterator is stateful so we use a Supplier factory method
+  private Supplier<BreakIterator> defaultBreakIterator = () -> BreakIterator.getSentenceInstance(Locale.ROOT);
+
+  private PassageScorer defaultScorer = new PassageScorer();
+
+  private PassageFormatter defaultFormatter = new DefaultPassageFormatter();
+
+  private int defaultMaxNoHighlightPassages = -1;
+
+  // lazy initialized with double-check locking; protected so subclass can init
+  protected volatile FieldInfos fieldInfos;
+
+  private int cacheFieldValCharsThreshold = DEFAULT_CACHE_CHARS_THRESHOLD;
+
+  /**
+   * Calls {@link Weight#extractTerms(Set)} on an empty index for the query.
+   */
+  protected static SortedSet<Term> extractTerms(Query query) throws IOException {
+    SortedSet<Term> queryTerms = new TreeSet<>();
+    EMPTY_INDEXSEARCHER.createNormalizedWeight(query, false).extractTerms(queryTerms);
+    return queryTerms;
+  }
+
+  /**
+   * Constructs the highlighter with the given index searcher and analyzer.
+   *
+   * @param indexSearcher Usually required, unless {@link #highlightWithoutSearcher(String, Query, String, int)} is
+   *                      used, in which case this needs to be null.
+   * @param indexAnalyzer Required, even if in some circumstances it isn't used.
+   */
+  public UnifiedHighlighter(IndexSearcher indexSearcher, Analyzer indexAnalyzer) {
+    this.searcher = indexSearcher; //TODO: make non nullable
+    this.indexAnalyzer = Objects.requireNonNull(indexAnalyzer,
+        "indexAnalyzer is required"
+            + " (even if in some circumstances it isn't used)");
+  }
+
+  public void setHandleMultiTermQuery(boolean handleMtq) {
+    this.defaultHandleMtq = handleMtq;
+  }
+
+  public void setHighlightPhrasesStrictly(boolean highlightPhrasesStrictly) {
+    this.defaultHighlightPhrasesStrictly = highlightPhrasesStrictly;
+  }
+
+  public void setMaxLength(int maxLength) {
+    if (maxLength < 0 || maxLength == Integer.MAX_VALUE) {
+      // two reasons: no overflow problems in BreakIterator.preceding(offset+1),
+      // our sentinel in the offsets queue uses this value to terminate.
+      throw new IllegalArgumentException("maxLength must be < Integer.MAX_VALUE");
+    }
+    this.maxLength = maxLength;
+  }
+
+  public void setBreakIterator(Supplier<BreakIterator> breakIterator) {
+    this.defaultBreakIterator = breakIterator;
+  }
+
+  public void setScorer(PassageScorer scorer) {
+    this.defaultScorer = scorer;
+  }
+
+  public void setFormatter(PassageFormatter formatter) {
+    this.defaultFormatter = formatter;
+  }
+
+  public void setMaxNoHighlightPassages(int defaultMaxNoHighlightPassages) {
+    this.defaultMaxNoHighlightPassages = defaultMaxNoHighlightPassages;
+  }
+
+  public void setCacheFieldValCharsThreshold(int cacheFieldValCharsThreshold) {
+    this.cacheFieldValCharsThreshold = cacheFieldValCharsThreshold;
+  }
+
+  /**
+   * Returns whether {@link MultiTermQuery} derivatives will be highlighted.  By default it's enabled.  MTQ
+   * highlighting can be expensive, particularly when using offsets in postings.
+   */
+  protected boolean shouldHandleMultiTermQuery(String field) {
+    return defaultHandleMtq;
+  }
+
+  /**
+   * Returns whether position sensitive queries (e.g. phrases and {@link SpanQuery}ies)
+   * should be highlighted strictly based on query matches (slower)
+   * versus any/all occurrences of the underlying terms.  By default it's enabled, but there's no overhead if such
+   * queries aren't used.
+   */
+  protected boolean shouldHighlightPhrasesStrictly(String field) {
+    return defaultHighlightPhrasesStrictly;
+  }
+
+  /**
+   * The maximum content size to process.  Content will be truncated to this size before highlighting. Typically
+   * snippets closer to the beginning of the document better summarize its content.
+   */
+  public int getMaxLength() {
+    return maxLength;
+  }
+
+  /**
+   * Returns the {@link BreakIterator} to use for
+   * dividing text into passages.  This returns
+   * {@link BreakIterator#getSentenceInstance(Locale)} by default;
+   * subclasses can override to customize.
+   * <p>
+   * Note: this highlighter will call
+   * {@link BreakIterator#preceding(int)} and {@link BreakIterator#next()} many times on it.
+   * The default generic JDK implementation of {@code preceding} performs poorly.
+   */
+  protected BreakIterator getBreakIterator(String field) {
+    return defaultBreakIterator.get();
+  }
+
+  /**
+   * Returns the {@link PassageScorer} to use for
+   * ranking passages.  This
+   * returns a new {@code PassageScorer} by default;
+   * subclasses can override to customize.
+   */
+  protected PassageScorer getScorer(String field) {
+    return defaultScorer;
+  }
+
+  /**
+   * Returns the {@link PassageFormatter} to use for
+   * formatting passages into highlighted snippets.  This
+   * returns a new {@code PassageFormatter} by default;
+   * subclasses can override to customize.
+   */
+  protected PassageFormatter getFormatter(String field) {
+    return defaultFormatter;
+  }
+
+  /**
+   * Returns the number of leading passages (as delineated by the {@link BreakIterator}) when no
+   * highlights could be found.  If it's less than 0 (the default) then this defaults to the {@code maxPassages}
+   * parameter given for each request.  If this is 0 then the resulting highlight is null (not formatted).
+   */
+  protected int getMaxNoHighlightPassages(String field) {
+    return defaultMaxNoHighlightPassages;
+  }
+
+  /**
+   * Limits the amount of field value pre-fetching until this threshold is passed.  The highlighter
+   * internally highlights in batches of documents sized on the sum field value length (in chars) of the fields
+   * to be highlighted (bounded by {@link #getMaxLength()} for each field).  By setting this to 0, you can force
+   * documents to be fetched and highlighted one at a time, which you usually shouldn't do.
+   * The default is 524288 chars which translates to about a megabyte.  However, note
+   * that the highlighter sometimes ignores this and highlights one document at a time (without caching a
+   * bunch of documents in advance) when it can detect there's no point in it -- such as when all fields will be
+   * highlighted via re-analysis as one example.
+   */
+  public int getCacheFieldValCharsThreshold() { // question: should we size by bytes instead?
+    return cacheFieldValCharsThreshold;
+  }
+
+  /**
+   * ... as passed in from constructor.
+   */
+  public IndexSearcher getIndexSearcher() {
+    return searcher;
+  }
+
+  /**
+   * ... as passed in from constructor.
+   */
+  public Analyzer getIndexAnalyzer() {
+    return indexAnalyzer;
+  }
+
+  /**
+   * Source of term offsets; essential for highlighting.
+   */
+  public enum OffsetSource {
+    POSTINGS, TERM_VECTORS, ANALYSIS, POSTINGS_WITH_TERM_VECTORS, NONE_NEEDED
+  }
+
+  /**
+   * Determine the offset source for the specified field.  The default algorithm is as follows:
+   * <ol>
+   * <li>This calls {@link #getFieldInfo(String)}. Note this returns null if there is no searcher or if the
+   * field isn't found there.</li>
+   * <li> If there's a field info it has
+   * {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS} then {@link OffsetSource#POSTINGS} is
+   * returned.</li>
+   * <li>If there's a field info and {@link FieldInfo#hasVectors()} then {@link OffsetSource#TERM_VECTORS} is
+   * returned (note we can't check here if the TV has offsets; if there isn't then an exception will get thrown
+   * down the line).</li>
+   * <li>Fall-back: {@link OffsetSource#ANALYSIS} is returned.</li>
+   * </ol>
+   * <p>
+   * Note that the highlighter sometimes switches to something else based on the query, such as if you have
+   * {@link OffsetSource#POSTINGS_WITH_TERM_VECTORS} but in fact don't need term vectors.
+   */
+  protected OffsetSource getOffsetSource(String field) {
+    FieldInfo fieldInfo = getFieldInfo(field);
+    if (fieldInfo != null) {
+      if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
+        return fieldInfo.hasVectors() ? OffsetSource.POSTINGS_WITH_TERM_VECTORS : OffsetSource.POSTINGS;
+      }
+      if (fieldInfo.hasVectors()) { // unfortunately we can't also check if the TV has offsets
+        return OffsetSource.TERM_VECTORS;
+      }
+    }
+    return OffsetSource.ANALYSIS;
+  }
+
+  /**
+   * Called by the default implementation of {@link #getOffsetSource(String)}.
+   * If there is no searcher then we simply always return null.
+   */
+  protected FieldInfo getFieldInfo(String field) {
+    if (searcher == null) {
+      return null;
+    }
+    // Need thread-safety for lazy-init but lets avoid 'synchronized' by using double-check locking idiom
+    FieldInfos fieldInfos = this.fieldInfos; // note: it's volatile; read once
+    if (fieldInfos == null) {
+      synchronized (this) {
+        fieldInfos = this.fieldInfos;
+        if (fieldInfos == null) {
+          fieldInfos = MultiFields.getMergedFieldInfos(searcher.getIndexReader());
+          this.fieldInfos = fieldInfos;
+        }
+
+      }
+
+    }
+    return fieldInfos.fieldInfo(field);
+  }
+
+  /**
+   * Highlights the top passages from a single field.
+   *
+   * @param field   field name to highlight.
+   *                Must have a stored string value and also be indexed with offsets.
+   * @param query   query to highlight.
+   * @param topDocs TopDocs containing the summary result documents to highlight.
+   * @return Array of formatted snippets corresponding to the documents in <code>topDocs</code>.
+   * If no highlights were found for a document, the
+   * first sentence for the field will be returned.
+   * @throws IOException              if an I/O error occurred during processing
+   * @throws IllegalArgumentException if <code>field</code> was indexed without
+   *                                  {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
+   */
+  public String[] highlight(String field, Query query, TopDocs topDocs) throws IOException {
+    return highlight(field, query, topDocs, 1);
+  }
+
+  /**
+   * Highlights the top-N passages from a single field.
+   *
+   * @param field       field name to highlight. Must have a stored string value.
+   * @param query       query to highlight.
+   * @param topDocs     TopDocs containing the summary result documents to highlight.
+   * @param maxPassages The maximum number of top-N ranked passages used to
+   *                    form the highlighted snippets.
+   * @return Array of formatted snippets corresponding to the documents in <code>topDocs</code>.
+   * If no highlights were found for a document, the
+   * first {@code maxPassages} sentences from the
+   * field will be returned.
+   * @throws IOException              if an I/O error occurred during processing
+   * @throws IllegalArgumentException if <code>field</code> was indexed without
+   *                                  {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
+   */
+  public String[] highlight(String field, Query query, TopDocs topDocs, int maxPassages) throws IOException {
+    Map<String, String[]> res = highlightFields(new String[]{field}, query, topDocs, new int[]{maxPassages});
+    return res.get(field);
+  }
+
+  /**
+   * Highlights the top passages from multiple fields.
+   * <p>
+   * Conceptually, this behaves as a more efficient form of:
+   * <pre class="prettyprint">
+   * Map m = new HashMap();
+   * for (String field : fields) {
+   * m.put(field, highlight(field, query, topDocs));
+   * }
+   * return m;
+   * </pre>
+   *
+   * @param fields  field names to highlight. Must have a stored string value.
+   * @param query   query to highlight.
+   * @param topDocs TopDocs containing the summary result documents to highlight.
+   * @return Map keyed on field name, containing the array of formatted snippets
+   * corresponding to the documents in <code>topDocs</code>.
+   * If no highlights were found for a document, the
+   * first sentence from the field will be returned.
+   * @throws IOException              if an I/O error occurred during processing
+   * @throws IllegalArgumentException if <code>field</code> was indexed without
+   *                                  {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
+   */
+  public Map<String, String[]> highlightFields(String[] fields, Query query, TopDocs topDocs) throws IOException {
+    int maxPassages[] = new int[fields.length];
+    Arrays.fill(maxPassages, 1);
+    return highlightFields(fields, query, topDocs, maxPassages);
+  }
+
+  /**
+   * Highlights the top-N passages from multiple fields.
+   * <p>
+   * Conceptually, this behaves as a more efficient form of:
+   * <pre class="prettyprint">
+   * Map m = new HashMap();
+   * for (String field : fields) {
+   * m.put(field, highlight(field, query, topDocs, maxPassages));
+   * }
+   * return m;
+   * </pre>
+   *
+   * @param fields      field names to highlight. Must have a stored string value.
+   * @param query       query to highlight.
+   * @param topDocs     TopDocs containing the summary result documents to highlight.
+   * @param maxPassages The maximum number of top-N ranked passages per-field used to
+   *                    form the highlighted snippets.
+   * @return Map keyed on field name, containing the array of formatted snippets
+   * corresponding to the documents in <code>topDocs</code>.
+   * If no highlights were found for a document, the
+   * first {@code maxPassages} sentences from the
+   * field will be returned.
+   * @throws IOException              if an I/O error occurred during processing
+   * @throws IllegalArgumentException if <code>field</code> was indexed without
+   *                                  {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
+   */
+  public Map<String, String[]> highlightFields(String[] fields, Query query, TopDocs topDocs, int[] maxPassages)
+      throws IOException {
+    final ScoreDoc scoreDocs[] = topDocs.scoreDocs;
+    int docids[] = new int[scoreDocs.length];
+    for (int i = 0; i < docids.length; i++) {
+      docids[i] = scoreDocs[i].doc;
+    }
+
+    return highlightFields(fields, query, docids, maxPassages);
+  }
+
+  /**
+   * Highlights the top-N passages from multiple fields,
+   * for the provided int[] docids.
+   *
+   * @param fieldsIn      field names to highlight. Must have a stored string value.
+   * @param query         query to highlight.
+   * @param docidsIn      containing the document IDs to highlight.
+   * @param maxPassagesIn The maximum number of top-N ranked passages per-field used to
+   *                      form the highlighted snippets.
+   * @return Map keyed on field name, containing the array of formatted snippets
+   * corresponding to the documents in <code>docidsIn</code>.
+   * If no highlights were found for a document, the
+   * first {@code maxPassages} from the field will
+   * be returned.
+   * @throws IOException              if an I/O error occurred during processing
+   * @throws IllegalArgumentException if <code>field</code> was indexed without
+   *                                  {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
+   */
+  public Map<String, String[]> highlightFields(String[] fieldsIn, Query query, int[] docidsIn, int[] maxPassagesIn)
+      throws IOException {
+    Map<String, String[]> snippets = new HashMap<>();
+    for (Map.Entry<String, Object[]> ent : highlightFieldsAsObjects(fieldsIn, query, docidsIn, maxPassagesIn).entrySet()) {
+      Object[] snippetObjects = ent.getValue();
+      String[] snippetStrings = new String[snippetObjects.length];
+      snippets.put(ent.getKey(), snippetStrings);
+      for (int i = 0; i < snippetObjects.length; i++) {
+        Object snippet = snippetObjects[i];
+        if (snippet != null) {
+          snippetStrings[i] = snippet.toString();
+        }
+      }
+    }
+
+    return snippets;
+  }
+
+  /**
+   * Expert: highlights the top-N passages from multiple fields,
+   * for the provided int[] docids, to custom Object as
+   * returned by the {@link PassageFormatter}.  Use
+   * this API to render to something other than String.
+   *
+   * @param fieldsIn      field names to highlight. Must have a stored string value.
+   * @param query         query to highlight.
+   * @param docIdsIn      containing the document IDs to highlight.
+   * @param maxPassagesIn The maximum number of top-N ranked passages per-field used to
+   *                      form the highlighted snippets.
+   * @return Map keyed on field name, containing the array of formatted snippets
+   * corresponding to the documents in <code>docIdsIn</code>.
+   * If no highlights were found for a document, the
+   * first {@code maxPassages} from the field will
+   * be returned.
+   * @throws IOException              if an I/O error occurred during processing
+   * @throws IllegalArgumentException if <code>field</code> was indexed without
+   *                                  {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}
+   */
+  protected Map<String, Object[]> highlightFieldsAsObjects(String[] fieldsIn, Query query, int[] docIdsIn,
+                                                           int[] maxPassagesIn) throws IOException {
+    if (fieldsIn.length < 1) {
+      throw new IllegalArgumentException("fieldsIn must not be empty");
+    }
+    if (fieldsIn.length != maxPassagesIn.length) {
+      throw new IllegalArgumentException("invalid number of maxPassagesIn");
+    }
+    if (searcher == null) {
+      throw new IllegalStateException("This method requires that an indexSearcher was passed in the "
+          + "constructor.  Perhaps you mean to call highlightWithoutSearcher?");
+    }
+
+    // Sort docs & fields for sequential i/o
+
+    // Sort doc IDs w/ index to original order: (copy input arrays since we sort in-place)
+    int[] docIds = new int[docIdsIn.length];
+    int[] docInIndexes = new int[docIds.length]; // fill in ascending order; points into docIdsIn[]
+    copyAndSortDocIdsWithIndex(docIdsIn, docIds, docInIndexes); // latter 2 are "out" params
+
+    // Sort fields w/ maxPassages pair: (copy input arrays since we sort in-place)
+    final String fields[] = new String[fieldsIn.length];
+    final int maxPassages[] = new int[maxPassagesIn.length];
+    copyAndSortFieldsWithMaxPassages(fieldsIn, maxPassagesIn, fields, maxPassages); // latter 2 are "out" params
+
+    // Init field highlighters (where most of the highlight logic lives, and on a per field basis)
+    SortedSet<Term> queryTerms = extractTerms(query);
+    FieldHighlighter[] fieldHighlighters = new FieldHighlighter[fields.length];
+    int numTermVectors = 0;
+    int numPostings = 0;
+    for (int f = 0; f < fields.length; f++) {
+      FieldHighlighter fieldHighlighter = getFieldHighlighter(fields[f], query, queryTerms, maxPassages[f]);
+      fieldHighlighters[f] = fieldHighlighter;
+
+      switch (fieldHighlighter.getOffsetSource()) {
+        case TERM_VECTORS:
+          numTermVectors++;
+          break;
+        case POSTINGS:
+          numPostings++;
+          break;
+        case POSTINGS_WITH_TERM_VECTORS:
+          numTermVectors++;
+          numPostings++;
+          break;
+        case ANALYSIS:
+        case NONE_NEEDED:
+        default:
+          //do nothing
+          break;
+      }
+    }
+
+    int cacheCharsThreshold = calculateOptimalCacheCharsThreshold(numTermVectors, numPostings);
+
+    IndexReader indexReaderWithTermVecCache =
+        (numTermVectors >= 2) ? TermVectorReusingLeafReader.wrap(searcher.getIndexReader()) : null;
+
+    // [fieldIdx][docIdInIndex] of highlightDoc result
+    Object[][] highlightDocsInByField = new Object[fields.length][docIds.length];
+    // Highlight in doc batches determined by loadFieldValues (consumes from docIdIter)
+    DocIdSetIterator docIdIter = asDocIdSetIterator(docIds);
+    for (int batchDocIdx = 0; batchDocIdx < docIds.length; ) {
+      // Load the field values of the first batch of document(s) (note: commonly all docs are in this batch)
+      List<CharSequence[]> fieldValsByDoc =
+          loadFieldValues(fields, docIdIter, cacheCharsThreshold);
+      //    the size of the above list is the size of the batch (num of docs in the batch)
+
+      // Highlight in per-field order first, then by doc (better I/O pattern)
+      for (int fieldIdx = 0; fieldIdx < fields.length; fieldIdx++) {
+        Object[] resultByDocIn = highlightDocsInByField[fieldIdx];//parallel to docIdsIn
+        FieldHighlighter fieldHighlighter = fieldHighlighters[fieldIdx];
+        for (int docIdx = batchDocIdx; docIdx - batchDocIdx < fieldValsByDoc.size(); docIdx++) {
+          int docId = docIds[docIdx];//sorted order
+          CharSequence content = fieldValsByDoc.get(docIdx - batchDocIdx)[fieldIdx];
+          if (content == null) {
+            continue;
+          }
+          IndexReader indexReader =
+              (fieldHighlighter.getOffsetSource() == OffsetSource.TERM_VECTORS
+                  && indexReaderWithTermVecCache != null)
+                  ? indexReaderWithTermVecCache
+                  : searcher.getIndexReader();
+          int docInIndex = docInIndexes[docIdx];//original input order
+          assert resultByDocIn[docInIndex] == null;
+          resultByDocIn[docInIndex] =
+              fieldHighlighter
+                  .highlightFieldForDoc(indexReader, docId, content.toString());
+        }
+
+      }
+
+      batchDocIdx += fieldValsByDoc.size();
+    }
+    assert docIdIter.docID() == DocIdSetIterator.NO_MORE_DOCS
+        || docIdIter.nextDoc() == DocIdSetIterator.NO_MORE_DOCS;
+
+    // TODO reconsider the return type; since this is an "advanced" method, lets not return a Map?  Notice the only
+    //    caller simply iterates it to build another structure.
+
+    // field -> object highlights parallel to docIdsIn
+    Map<String, Object[]> resultMap = new HashMap<>(fields.length);
+    for (int f = 0; f < fields.length; f++) {
+      resultMap.put(fields[f], highlightDocsInByField[f]);
+    }
+    return resultMap;
+  }
+
+  /**
+   * When cacheCharsThreshold is 0, loadFieldValues() only fetches one document at a time.  We override it to be 0
+   * in two circumstances:
+   */
+  private int calculateOptimalCacheCharsThreshold(int numTermVectors, int numPostings) {
+    if (numPostings == 0 && numTermVectors == 0) {
+      // (1) When all fields are ANALYSIS there's no point in caching a batch of documents
+      // because no other info on disk is needed to highlight it.
+      return 0;
+    } else if (numTermVectors >= 2) {
+      // (2) When two or more fields have term vectors, given the field-then-doc algorithm, the underlying term
+      // vectors will be fetched in a terrible access pattern unless we highlight a doc at a time and use a special
+      // current-doc TV cache.  So we do that.  Hopefully one day TVs will be improved to make this pointless.
+      return 0;
+    } else {
+      return getCacheFieldValCharsThreshold();
+    }
+  }
+
+  private void copyAndSortFieldsWithMaxPassages(String[] fieldsIn, int[] maxPassagesIn, final String[] fields,
+                                                final int[] maxPassages) {
+    System.arraycopy(fieldsIn, 0, fields, 0, fieldsIn.length);
+    System.arraycopy(maxPassagesIn, 0, maxPassages, 0, maxPassagesIn.length);
+    new InPlaceMergeSorter() {
+      @Override
+      protected void swap(int i, int j) {
+        String tmp = fields[i];
+        fields[i] = fields[j];
+        fields[j] = tmp;
+        int tmp2 = maxPassages[i];
+        maxPassages[i] = maxPassages[j];
+        maxPassages[j] = tmp2;
+      }
+
+      @Override
+      protected int compare(int i, int j) {
+        return fields[i].compareTo(fields[j]);
+      }
+
+    }.sort(0, fields.length);
+  }
+
+  private void copyAndSortDocIdsWithIndex(int[] docIdsIn, final int[] docIds, final int[] docInIndexes) {
+    System.arraycopy(docIdsIn, 0, docIds, 0, docIdsIn.length);
+    for (int i = 0; i < docInIndexes.length; i++) {
+      docInIndexes[i] = i;
+    }
+    new InPlaceMergeSorter() {
+      @Override
+      protected void swap(int i, int j) {
+        int tmp = docIds[i];
+        docIds[i] = docIds[j];
+        docIds[j] = tmp;
+        tmp = docInIndexes[i];
+        docInIndexes[i] = docInIndexes[j];
+        docInIndexes[j] = tmp;
+      }
+
+      @Override
+      protected int compare(int i, int j) {
+        return Integer.compare(docIds[i], docIds[j]);
+      }
+    }.sort(0, docIds.length);
+  }
+
+  /**
+   * Highlights text passed as a parameter.  This requires the {@link IndexSearcher} provided to this highlighter is
+   * null.  This use-case is more rare.  Naturally, the mode of operation will be {@link OffsetSource#ANALYSIS}.
+   * The result of this method is whatever the {@link PassageFormatter} returns.  For the {@link
+   * DefaultPassageFormatter} and assuming {@code content} has non-zero length, the result will be a non-null
+   * string -- so it's safe to call {@link Object#toString()} on it in that case.
+   *
+   * @param field       field name to highlight (as found in the query).
+   * @param query       query to highlight.
+   * @param content     text to highlight.
+   * @param maxPassages The maximum number of top-N ranked passages used to
+   *                    form the highlighted snippets.
+   * @return result of the {@link PassageFormatter} -- probably a String.  Might be null.
+   * @throws IOException if an I/O error occurred during processing
+   */
+  //TODO make content a List? and return a List? and ensure getEmptyHighlight is never invoked multiple times?
+  public Object highlightWithoutSearcher(String field, Query query, String content, int maxPassages)
+      throws IOException {
+    if (this.searcher != null) {
+      throw new IllegalStateException("highlightWithoutSearcher should only be called on a " +
+          getClass().getSimpleName() + " without an IndexSearcher.");
+    }
+    Objects.requireNonNull(content, "content is required");
+    SortedSet<Term> queryTerms = extractTerms(query);
+    return getFieldHighlighter(field, query, queryTerms, maxPassages)
+        .highlightFieldForDoc(null, -1, content);
+  }
+
+  protected FieldHighlighter getFieldHighlighter(String field, Query query, SortedSet<Term> allTerms, int maxPassages) {
+    return new FieldHighlighter(field,
+        getOffsetStrategy(field, query, allTerms),
+        new SplittingBreakIterator(getBreakIterator(field), UnifiedHighlighter.MULTIVAL_SEP_CHAR),
+        getScorer(field),
+        maxPassages,
+        getMaxNoHighlightPassages(field),
+        getFormatter(field));
+  }
+
+  protected FieldOffsetStrategy getOffsetStrategy(String field, Query query, SortedSet<Term> allTerms) {
+    EnumSet<HighlightFlag> highlightFlags = getFlags(field);
+    BytesRef[] terms = filterExtractedTerms(field, allTerms);
+    PhraseHelper phraseHelper = getPhraseHelper(field, query, highlightFlags);
+    CharacterRunAutomaton[] automata = getAutomata(field, query, highlightFlags);
+    OffsetSource offsetSource = getOptimizedOffsetSource(field, terms, phraseHelper, automata);
+    switch (offsetSource) {
+      case ANALYSIS:
+        return new AnalysisOffsetStrategy(field, terms, phraseHelper, automata, getIndexAnalyzer());
+      case NONE_NEEDED:
+        return NoOpOffsetStrategy.INSTANCE;
+      case TERM_VECTORS:
+        return new TermVectorOffsetStrategy(field, terms, phraseHelper, automata);
+      case POSTINGS:
+        return new PostingsOffsetStrategy(field, terms, phraseHelper, automata);
+      case POSTINGS_WITH_TERM_VECTORS:
+        return new PostingsWithTermVectorsOffsetStrategy(field, terms, phraseHelper, automata);
+      default:
+        throw new IllegalArgumentException("Unrecognized offset source " + offsetSource);
+    }
+  }
+
+  protected EnumSet<HighlightFlag> getFlags(String field) {
+    EnumSet<HighlightFlag> highlightFlags = EnumSet.noneOf(HighlightFlag.class);
+    if (shouldHandleMultiTermQuery(field)) {
+      highlightFlags.add(HighlightFlag.MULTI_TERM_QUERY);
+    }
+    if (shouldHighlightPhrasesStrictly(field)) {
+      highlightFlags.add(HighlightFlag.PHRASES);
+    }
+    return highlightFlags;
+  }
+
+  protected BytesRef[] filterExtractedTerms(String field, SortedSet<Term> queryTerms) {
+    // TODO consider requireFieldMatch
+    Term floor = new Term(field, "");
+    Term ceiling = new Term(field, UnicodeUtil.BIG_TERM);
+    SortedSet<Term> fieldTerms = queryTerms.subSet(floor, ceiling);
+
+    // Strip off the redundant field:
+    BytesRef[] terms = new BytesRef[fieldTerms.size()];
+    int termUpto = 0;
+    for (Term term : fieldTerms) {
+      terms[termUpto++] = term.bytes();
+    }
+    return terms;
+  }
+
+  protected PhraseHelper getPhraseHelper(String field, Query query, EnumSet<HighlightFlag> highlightFlags) {
+    boolean highlightPhrasesStrictly = highlightFlags.contains(HighlightFlag.PHRASES);
+    boolean handleMultiTermQuery = highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY);
+    return highlightPhrasesStrictly ?
+        new PhraseHelper(query, field, this::requiresRewrite, !handleMultiTermQuery) :
+        PhraseHelper.NONE;
+  }
+
+  protected CharacterRunAutomaton[] getAutomata(String field, Query query, EnumSet<HighlightFlag> highlightFlags) {
+    return highlightFlags.contains(HighlightFlag.MULTI_TERM_QUERY)
+        ? MultiTermHighlighting.extractAutomata(query, field, !highlightFlags.contains(HighlightFlag.PHRASES))
+        : ZERO_LEN_AUTOMATA_ARRAY;
+  }
+
+  protected OffsetSource getOptimizedOffsetSource(String field, BytesRef[] terms, PhraseHelper phraseHelper, CharacterRunAutomaton[] automata) {
+    if (terms.length == 0 && automata.length == 0 && !phraseHelper.willRewrite()) {
+      return OffsetSource.NONE_NEEDED; //nothing to highlight
+    }
+
+    OffsetSource offsetSource = getOffsetSource(field);
+    switch (offsetSource) {
+      case POSTINGS:
+        if (phraseHelper.willRewrite()) {
+          // We can't choose the postings offset source when there is "rewriting" in the strict phrase
+          // processing (rare but possible). Postings requires knowing all the terms (except wildcards)
+          // up front.
+          return OffsetSource.ANALYSIS;
+        } else if (automata.length > 0) {
+          return OffsetSource.ANALYSIS;
+        }
+        break;
+      case POSTINGS_WITH_TERM_VECTORS:
+        if (!phraseHelper.willRewrite() && automata.length == 0) {
+          return OffsetSource.POSTINGS; //We don't need term vectors
+        }
+        break;
+      case ANALYSIS:
+      case TERM_VECTORS:
+      case NONE_NEEDED:
+      default:
+        //stick with the original offset source
+        break;
+    }
+
+    return offsetSource;
+  }
+
+  /**
+   * When highlighting phrases accurately, we need to know which {@link SpanQuery}'s need to have
+   * {@link Query#rewrite(IndexReader)} called on them.  It helps performance to avoid it if it's not needed.
+   * This method will be invoked on all SpanQuery instances recursively. If you have custom SpanQuery queries then
+   * override this to check instanceof and provide a definitive answer. If the query isn't your custom one, simply
+   * return null to have the default rules apply, which govern the ones included in Lucene.
+   */
+  protected Boolean requiresRewrite(SpanQuery spanQuery) {
+    return null;
+  }
+
+  private DocIdSetIterator asDocIdSetIterator(int[] sortedDocIds) {
+    return new DocIdSetIterator() {
+      int idx = -1;
+
+      @Override
+      public int docID() {
+        if (idx < 0 || idx >= sortedDocIds.length) {
+          return NO_MORE_DOCS;
+        }
+        return sortedDocIds[idx];
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        idx++;
+        return docID();
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        return super.slowAdvance(target); // won't be called, so whatever
+      }
+
+      @Override
+      public long cost() {
+        return Math.max(0, sortedDocIds.length - (idx + 1)); // remaining docs
+      }
+    };
+  }
+
+  /**
+   * Loads the String values for each docId by field to be highlighted.  By default this loads from stored fields
+   * by the same name as given, but a subclass can change the source.  The returned Strings must be identical to
+   * what was indexed (at least for postings or term-vectors offset sources).
+   * This method must load fields for at least one document from the given {@link DocIdSetIterator}
+   * but need not return all of them; by default the character lengths are summed and this method will return early
+   * when {@code cacheCharsThreshold} is exceeded.  Specifically if that number is 0, then only one document is
+   * fetched no matter what.  Values in the array of {@link CharSequence} will be null if no value was found.
+   */
+  protected List<CharSequence[]> loadFieldValues(String[] fields,
+                                                 DocIdSetIterator docIter, int cacheCharsThreshold)
+      throws IOException {
+    List<CharSequence[]> docListOfFields =
+        new ArrayList<>(cacheCharsThreshold == 0 ? 1 : (int) Math.min(64, docIter.cost()));
+
+    LimitedStoredFieldVisitor visitor = newLimitedStoredFieldsVisitor(fields);
+    int sumChars = 0;
+    do {
+      int docId = docIter.nextDoc();
+      if (docId == DocIdSetIterator.NO_MORE_DOCS) {
+        break;
+      }
+      visitor.init();
+      searcher.doc(docId, visitor);
+      CharSequence[] valuesByField = visitor.getValuesByField();
+      docListOfFields.add(valuesByField);
+      for (CharSequence val : valuesByField) {
+        sumChars += (val == null ? 0 : val.length());
+      }
+    } while (sumChars <= cacheCharsThreshold && cacheCharsThreshold != 0);
+    return docListOfFields;
+  }
+
+  /**
+   * @lucene.internal
+   */
+  protected LimitedStoredFieldVisitor newLimitedStoredFieldsVisitor(String[] fields) {
+    return new LimitedStoredFieldVisitor(fields, MULTIVAL_SEP_CHAR, getMaxLength());
+  }
+
+  /**
+   * Fetches stored fields for highlighting. Uses a multi-val separator char and honors a max length to retrieve.
+   * @lucene.internal
+   */
+  protected static class LimitedStoredFieldVisitor extends StoredFieldVisitor {
+    protected final String[] fields;
+    protected final char valueSeparator;
+    protected final int maxLength;
+    protected CharSequence[] values;// starts off as String; may become StringBuilder.
+    protected int currentField;
+
+    public LimitedStoredFieldVisitor(String[] fields, char valueSeparator, int maxLength) {
+      this.fields = fields;
+      this.valueSeparator = valueSeparator;
+      this.maxLength = maxLength;
+    }
+
+    void init() {
+      values = new CharSequence[fields.length];
+      currentField = -1;
+    }
+
+    @Override
+    public void stringField(FieldInfo fieldInfo, byte[] byteValue) throws IOException {
+      String value = new String(byteValue, StandardCharsets.UTF_8);
+      assert currentField >= 0;
+      CharSequence curValue = values[currentField];
+      if (curValue == null) {
+        //question: if truncate due to maxLength, should we try and avoid keeping the other chars in-memory on
+        //  the backing char[]?
+        values[currentField] = value.substring(0, Math.min(maxLength, value.length()));//note: may return 'this'
+        return;
+      }
+      final int lengthBudget = maxLength - curValue.length();
+      if (lengthBudget <= 0) {
+        return;
+      }
+      StringBuilder curValueBuilder;
+      if (curValue instanceof StringBuilder) {
+        curValueBuilder = (StringBuilder) curValue;
+      } else {
+        // upgrade String to StringBuilder. Choose a good initial size.
+        curValueBuilder = new StringBuilder(curValue.length() + Math.min(lengthBudget, value.length() + 256));
+        curValueBuilder.append(curValue);
+      }
+      curValueBuilder.append(valueSeparator);
+      curValueBuilder.append(value.substring(0, Math.min(lengthBudget - 1, value.length())));
+      values[currentField] = curValueBuilder;
+    }
+
+    @Override
+    public Status needsField(FieldInfo fieldInfo) throws IOException {
+      currentField = Arrays.binarySearch(fields, fieldInfo.name);
+      if (currentField < 0) {
+        return Status.NO;
+      }
+      CharSequence curVal = values[currentField];
+      if (curVal != null && curVal.length() >= maxLength) {
+        return fields.length == 1 ? Status.STOP : Status.NO;
+      }
+      return Status.YES;
+    }
+
+    CharSequence[] getValuesByField() {
+      return this.values;
+    }
+
+  }
+
+  /**
+   * Wraps an IndexReader that remembers/caches the last call to {@link LeafReader#getTermVectors(int)} so that
+   * if the next call has the same ID, then it is reused.  If TV's were column-stride (like doc-values), there would
+   * be no need for this.
+   */
+  private static class TermVectorReusingLeafReader extends FilterLeafReader {
+
+    static IndexReader wrap(IndexReader reader) throws IOException {
+      LeafReader[] leafReaders = reader.leaves().stream()
+          .map(LeafReaderContext::reader)
+          .map(TermVectorReusingLeafReader::new)
+          .toArray(LeafReader[]::new);
+      return new BaseCompositeReader<IndexReader>(leafReaders) {
+        @Override
+        protected void doClose() throws IOException {
+          reader.close();
+        }
+      };
+    }
+
+    private int lastDocId = -1;
+    private Fields tvFields;
+
+    TermVectorReusingLeafReader(LeafReader in) {
+      super(in);
+    }
+
+    @Override
+    public Fields getTermVectors(int docID) throws IOException {
+      if (docID != lastDocId) {
+        lastDocId = docID;
+        tvFields = in.getTermVectors(docID);
+      }
+      return tvFields;
+    }
+
+  }
+
+  /**
+   * Flags for controlling highlighting behavior.
+   */
+  public enum HighlightFlag {
+    PHRASES,
+    MULTI_TERM_QUERY
+    // TODO: ignoreQueryFields
+    // TODO: useQueryBoosts
+    // TODO: avoidMemoryIndexIfPossible
+    // TODO: preferMemoryIndexForStats
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/package-info.java b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/package-info.java
new file mode 100644
index 0000000..b0acd36
--- /dev/null
+++ b/lucene/highlighter/src/java/org/apache/lucene/search/uhighlight/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * The UnifiedHighlighter -- a flexible highlighter that can get offsets from postings, term vectors, or analysis.
+ * @see org.apache.lucene.search.uhighlight.UnifiedHighlighter
+ */
+package org.apache.lucene.search.uhighlight;


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

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestSplittingBreakIterator.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestSplittingBreakIterator.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestSplittingBreakIterator.java
new file mode 100644
index 0000000..b78f329
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestSplittingBreakIterator.java
@@ -0,0 +1,192 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestSplittingBreakIterator extends LuceneTestCase {
+
+
+  private static final BreakIterator LINE_BI = BreakIterator.getLineInstance(Locale.ROOT);
+  private static final BreakIterator SPLIT_BI = new SplittingBreakIterator(LINE_BI, '|');
+
+  public void testLineBreakIterator() {
+    testWithoutSplits(LINE_BI);
+  }
+
+  private void testWithoutSplits(BreakIterator bi) {
+    // these tests have no '|'
+    testBreakIterator(bi,
+        " a",
+        "^^^");
+    testBreakIterator(bi,
+        "aa",
+        "^ ^");
+    testBreakIterator(bi,
+        "aa a",
+        "^  ^^");
+  }
+
+  public void testWithoutSplits() {
+    testWithoutSplits(SPLIT_BI);
+  }
+
+  public void testOnlySingleSplitChar() {
+    testBreakIterator(SPLIT_BI,
+        "|",
+        "^^");
+  }
+
+  public void testSplitThenValue() {
+    testBreakIterator(SPLIT_BI,
+        "|a",
+        "^^^");
+  }
+
+  public void testValueThenSplit() {
+    testBreakIterator(SPLIT_BI,
+        "a|",
+        "^^^");
+  }
+
+  public void testValueThenSplitThenValue() {
+    testBreakIterator(SPLIT_BI,
+        "aa|aa",
+        "^ ^^ ^");
+  }
+
+  public void testValueThenDoubleSplitThenValue() {
+    testBreakIterator(SPLIT_BI,
+        "aa||aa",
+        "^ ^^^ ^");
+  }
+
+  public void testValueThenSplitThenDoubleValueThenSplitThenValue() {
+    testBreakIterator(SPLIT_BI,
+        "a|bb cc|d",
+        "^^^  ^ ^^^");
+  }
+
+  private void testBreakIterator(BreakIterator bi, String text, String boundaries) {
+    bi.setText(text);
+
+    //Test first & last
+    testFirstAndLast(bi, text, boundaries);
+
+    //Test if expected boundaries are consistent with reading them from next() in a loop:
+    assertEquals(boundaries, readBoundariesToString(bi, text));
+
+    //Test following() and preceding():
+    // get each index, randomized in case their is a sequencing bug:
+    List<Integer> indexes = randomIntsBetweenInclusive(text.length() + 1);
+    testFollowing(bi, text, boundaries, indexes);
+    testPreceding(bi, text, boundaries, indexes);
+
+    //Test previous():
+    testPrevious(bi, text, boundaries);
+  }
+
+  private void testFirstAndLast(BreakIterator bi, String text, String boundaries) {
+    String message = "Text: " + text;
+    int current = bi.current();
+    assertEquals(message, boundaries.indexOf('^'), current);
+    assertEquals(message, current, bi.first());
+    assertEquals(message, current, bi.current());
+    current = bi.last();
+    assertEquals(boundaries.lastIndexOf('^'), current);
+    assertEquals(message, current, bi.current());
+  }
+
+  private void testFollowing(BreakIterator bi, String text, String boundaries, List<Integer> indexes) {
+    String message = "Text: " + text;
+    for (Integer index : indexes) {
+      int got = bi.following(index);
+      if (index == boundaries.length()) {
+        assertEquals(message, BreakIterator.DONE, got);
+        assertEquals(boundaries.lastIndexOf('^'), bi.current());
+        continue;
+      }
+      assertEquals(message + " index:" + index, boundaries.indexOf('^', index + 1), got);
+    }
+  }
+
+  private void testPreceding(BreakIterator bi, String text, String boundaries, List<Integer> indexes) {
+    String message = "Text: " + text;
+    for (Integer index : indexes) {
+      int got = bi.preceding(index);
+      if (index == 0) {
+        assertEquals(message, BreakIterator.DONE, got);
+        assertEquals(boundaries.indexOf('^'), bi.current());
+        continue;
+      }
+//            if (index == text.length() && got == BreakIterator.DONE) {
+//                continue;//hack to accept faulty default impl of BreakIterator.preceding()
+//            }
+      assertEquals(message + " index:" + index, boundaries.lastIndexOf('^', index - 1), got);
+    }
+  }
+
+  private List<Integer> randomIntsBetweenInclusive(int end) {
+    List<Integer> indexes = new ArrayList<>(end);
+    for (int i = 0; i < end; i++) {
+      indexes.add(i);
+    }
+    Collections.shuffle(indexes, random());
+    return indexes;
+  }
+
+  private void testPrevious(BreakIterator bi, String text, String boundaries) {
+    String message = "Text: " + text;
+
+    bi.setText(text);
+    int idx = bi.last();//position at the end
+    while (true) {
+      idx = boundaries.lastIndexOf('^', idx - 1);
+      if (idx == -1) {
+        assertEquals(message, BreakIterator.DONE, bi.previous());
+        break;
+      }
+      assertEquals(message, idx, bi.previous());
+    }
+    assertEquals(message, boundaries.indexOf('^'), bi.current());//finishes at first
+  }
+
+  /**
+   * Returns a string comprised of spaces and '^' only at the boundaries.
+   */
+  private String readBoundariesToString(BreakIterator bi, String text) {
+    // init markers to spaces
+    StringBuilder markers = new StringBuilder();
+    markers.setLength(text.length() + 1);
+    for (int k = 0; k < markers.length(); k++) {
+      markers.setCharAt(k, ' ');
+    }
+
+    bi.setText(text);
+    for (int boundary = bi.current(); boundary != BreakIterator.DONE; boundary = bi.next()) {
+      markers.setCharAt(boundary, '^');
+    }
+    return markers.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
new file mode 100644
index 0000000..0fd7d3d
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighter.java
@@ -0,0 +1,962 @@
+/*
+ * 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.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.text.BreakIterator;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.postingshighlight.WholeBreakIterator;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.junit.After;
+import org.junit.Before;
+
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestUnifiedHighlighter extends LuceneTestCase {
+
+  private final FieldType fieldType; // for "body" generally, but not necessarily others. See constructor
+
+  private MockAnalyzer indexAnalyzer;
+  private Directory dir;
+
+  @ParametersFactory
+  public static Iterable<Object[]> parameters() {
+    return UHTestHelper.parametersFactoryList();
+  }
+
+  public TestUnifiedHighlighter(FieldType fieldType) {
+    this.fieldType = fieldType;
+  }
+
+  @Before
+  public void doBefore() throws IOException {
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
+    dir = newDirectory();
+  }
+
+  @After
+  public void doAfter() throws IOException {
+    dir.close();
+  }
+
+  //
+  //  Tests below were ported from the PostingsHighlighter. Possibly augmented.  Far below are newer tests.
+  //
+
+  public void testBasics() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+    body.setStringValue("Highlighting the first term. Hope it works.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("Just a test <b>highlighting</b> from postings. ", snippets[0]);
+    assertEquals("<b>Highlighting</b> the first term. ", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testFormatWithMatchExceedingContentLength2() throws Exception {
+
+    String bodyText = "123 TEST 01234 TEST";
+
+    String[] snippets = formatWithMatchExceedingContentLength(bodyText);
+
+    assertEquals(1, snippets.length);
+    assertEquals("123 <b>TEST</b> 01234 TE", snippets[0]);
+  }
+
+  public void testFormatWithMatchExceedingContentLength3() throws Exception {
+
+    String bodyText = "123 5678 01234 TEST TEST";
+
+    String[] snippets = formatWithMatchExceedingContentLength(bodyText);
+
+    assertEquals(1, snippets.length);
+    assertEquals("123 5678 01234 TE", snippets[0]);
+  }
+
+  public void testFormatWithMatchExceedingContentLength() throws Exception {
+
+    String bodyText = "123 5678 01234 TEST";
+
+    String[] snippets = formatWithMatchExceedingContentLength(bodyText);
+
+    assertEquals(1, snippets.length);
+    // LUCENE-5166: no snippet
+    assertEquals("123 5678 01234 TE", snippets[0]);
+  }
+
+  private String[] formatWithMatchExceedingContentLength(String bodyText) throws IOException {
+
+    int maxLength = 17;
+
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    final Field body = new Field("body", bodyText, fieldType);
+
+    Document doc = new Document();
+    doc.add(body);
+
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+
+    Query query = new TermQuery(new Term("body", "test"));
+
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setMaxLength(maxLength);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+
+
+    ir.close();
+    return snippets;
+  }
+
+  // simple test highlighting last word.
+  public void testHighlightLastWord() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(1, snippets.length);
+    assertEquals("This is a <b>test</b>", snippets[0]);
+
+    ir.close();
+  }
+
+  // simple test with one sentence documents.
+  public void testOneSentence() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  // simple test with multiple values that make a result longer than maxLength.
+  public void testMaxLengthWithMultivalue() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+
+    final String value = "This is a multivalued field. Sentencetwo field.";
+    doc.add(new Field("body", value, fieldType));
+    doc.add(new Field("body", value, fieldType));
+    doc.add(new Field("body", value, fieldType));
+
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setMaxLength(value.length() * 2 + 1);
+    Query query = new TermQuery(new Term("body", "field"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 10);
+    assertEquals(1, snippets.length);
+    String highlightedValue = "This is a multivalued <b>field</b>. Sentencetwo <b>field</b>.";
+    assertEquals(highlightedValue + "... " + highlightedValue, snippets[0]);
+
+    ir.close();
+  }
+
+  public void testMultipleFields() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Field title = new Field("title", "", UHTestHelper.randomFieldType(random()));
+    Document doc = new Document();
+    doc.add(body);
+    doc.add(title);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    title.setStringValue("I am hoping for the best.");
+    iw.addDocument(doc);
+    body.setStringValue("Highlighting the first term. Hope it works.");
+    title.setStringValue("But best may not be good enough.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("title", "best")), BooleanClause.Occur.SHOULD)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    Map<String, String[]> snippets = highlighter.highlightFields(new String[]{"body", "title"}, query, topDocs);
+    assertEquals(2, snippets.size());
+    assertEquals("Just a test <b>highlighting</b> from postings. ", snippets.get("body")[0]);
+    assertEquals("<b>Highlighting</b> the first term. ", snippets.get("body")[1]);
+    assertEquals("I am hoping for the <b>best</b>.", snippets.get("title")[0]);
+    assertEquals("But <b>best</b> may not be good enough.", snippets.get("title")[1]);
+    ir.close();
+  }
+
+  public void testMultipleTerms() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+    body.setStringValue("Highlighting the first term. Hope it works.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "highlighting")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("body", "just")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("body", "first")), BooleanClause.Occur.SHOULD)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("<b>Just</b> a test <b>highlighting</b> from postings. ", snippets[0]);
+    assertEquals("<b>Highlighting</b> the <b>first</b> term. ", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testMultiplePassages() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+    body.setStringValue("This test is another test. Not a good sentence. Test test test test.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>. Just a <b>test</b> highlighting from postings. ", snippets[0]);
+    assertEquals("This <b>test</b> is another <b>test</b>. ... <b>Test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testBuddhism() throws Exception {
+    String text = "This eight-volume set brings together seminal papers in Buddhist studies from a vast " +
+        "range of academic disciplines published over the last forty years. With a new introduction " +
+        "by the editor, this collection is a unique and unrivalled research resource for both " +
+        "student and scholar. Coverage includes: - Buddhist origins; early history of Buddhism in " +
+        "South and Southeast Asia - early Buddhist Schools and Doctrinal History; Theravada Doctrine " +
+        "- the Origins and nature of Mahayana Buddhism; some Mahayana religious topics - Abhidharma " +
+        "and Madhyamaka - Yogacara, the Epistemological tradition, and Tathagatagarbha - Tantric " +
+        "Buddhism (Including China and Japan); Buddhism in Nepal and Tibet - Buddhism in South and " +
+        "Southeast Asia, and - Buddhism in China, East Asia, and Japan.";
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", text, fieldType);
+    Document document = new Document();
+    document.add(body);
+    iw.addDocument(document);
+    IndexReader ir = iw.getReader();
+    iw.close();
+    IndexSearcher searcher = newSearcher(ir);
+    PhraseQuery query = new PhraseQuery.Builder()
+        .add(new Term("body", "buddhist"))
+        .add(new Term("body", "origins"))
+        .build();
+    TopDocs topDocs = searcher.search(query, 10);
+    assertEquals(1, topDocs.totalHits);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setHighlightPhrasesStrictly(false);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertTrue(snippets[0].contains("<b>Buddhist</b> <b>origins</b>"));
+    ir.close();
+  }
+
+  public void testCuriousGeorge() throws Exception {
+    String text = "It\u2019s the formula for success for preschoolers\u2014Curious George and fire trucks! " +
+        "Curious George and the Firefighters is a story based on H. A. and Margret Rey\u2019s " +
+        "popular primate and painted in the original watercolor and charcoal style. " +
+        "Firefighters are a famously brave lot, but can they withstand a visit from one curious monkey?";
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", text, fieldType);
+    Document document = new Document();
+    document.add(body);
+    iw.addDocument(document);
+    IndexReader ir = iw.getReader();
+    iw.close();
+    IndexSearcher searcher = newSearcher(ir);
+    PhraseQuery query = new PhraseQuery.Builder()
+        .add(new Term("body", "curious"))
+        .add(new Term("body", "george"))
+        .build();
+    TopDocs topDocs = searcher.search(query, 10);
+    assertEquals(1, topDocs.totalHits);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setHighlightPhrasesStrictly(false);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertFalse(snippets[0].contains("<b>Curious</b>Curious"));
+    ir.close();
+  }
+
+  public void testCambridgeMA() throws Exception {
+    BufferedReader r = new BufferedReader(new InputStreamReader(
+        this.getClass().getResourceAsStream("CambridgeMA.utf8"), StandardCharsets.UTF_8));
+    String text = r.readLine();
+    r.close();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+    Field body = new Field("body", text, fieldType);
+    Document document = new Document();
+    document.add(body);
+    iw.addDocument(document);
+    IndexReader ir = iw.getReader();
+    iw.close();
+    IndexSearcher searcher = newSearcher(ir);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "porter")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("body", "square")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("body", "massachusetts")), BooleanClause.Occur.SHOULD)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10);
+    assertEquals(1, topDocs.totalHits);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setMaxLength(Integer.MAX_VALUE - 1);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertTrue(snippets[0].contains("<b>Square</b>"));
+    assertTrue(snippets[0].contains("<b>Porter</b>"));
+    ir.close();
+  }
+
+  public void testPassageRanking() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.  Just highlighting from postings. This is also a much sillier test.  Feel free to test test test test test test test.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertEquals("This is a <b>test</b>.  ... Feel free to <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[0]);
+
+    ir.close();
+  }
+
+  public void testBooleanMustNot() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "This sentence has both terms.  This sentence has only terms.", fieldType);
+    Document document = new Document();
+    document.add(body);
+    iw.addDocument(document);
+    IndexReader ir = iw.getReader();
+    iw.close();
+    IndexSearcher searcher = newSearcher(ir);
+
+    BooleanQuery query2 = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "both")), BooleanClause.Occur.MUST_NOT)
+        .build();
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "terms")), BooleanClause.Occur.SHOULD)
+        .add(query2, BooleanClause.Occur.SHOULD)
+        .build();
+
+    TopDocs topDocs = searcher.search(query, 10);
+    assertEquals(1, topDocs.totalHits);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setMaxLength(Integer.MAX_VALUE - 1);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertFalse(snippets[0].contains("<b>both</b>"));
+    ir.close();
+  }
+
+  public void testHighlightAllText() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.  Just highlighting from postings. This is also a much sillier test.  Feel free to test test test test test test test.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected BreakIterator getBreakIterator(String field) {
+        return new WholeBreakIterator();
+      }
+    };
+    highlighter.setMaxLength(10000);
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertEquals("This is a <b>test</b>.  Just highlighting from postings. This is also a much sillier <b>test</b>.  Feel free to <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[0]);
+
+    ir.close();
+  }
+
+  public void testSpecificDocIDs() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+    body.setStringValue("Highlighting the first term. Hope it works.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    ScoreDoc[] hits = topDocs.scoreDocs;
+    int[] docIDs = new int[2];
+    docIDs[0] = hits[0].doc;
+    docIDs[1] = hits[1].doc;
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{1}).get("body");
+    assertEquals(2, snippets.length);
+    assertEquals("Just a test <b>highlighting</b> from postings. ", snippets[0]);
+    assertEquals("<b>Highlighting</b> the first term. ", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testCustomFieldValueSource() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+
+    final String text = "This is a test.  Just highlighting from postings. This is also a much sillier test.  Feel free to test test test test test test test.";
+    Field body = new Field("body", text, fieldType);
+    doc.add(body);
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected List<CharSequence[]> loadFieldValues(String[] fields,
+                                                     DocIdSetIterator docIter, int cacheCharsThreshold) throws IOException {
+        assert fields.length == 1;
+        assert docIter.cost() == 1;
+        docIter.nextDoc();
+        return Collections.singletonList(new CharSequence[]{text});
+      }
+
+      @Override
+      protected BreakIterator getBreakIterator(String field) {
+        return new WholeBreakIterator();
+      }
+    };
+
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+    assertEquals(1, snippets.length);
+    assertEquals("This is a <b>test</b>.  Just highlighting from postings. This is also a much sillier <b>test</b>.  Feel free to <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b> <b>test</b>.", snippets[0]);
+
+    ir.close();
+  }
+
+  /**
+   * Make sure highlighter returns first N sentences if
+   * there were no hits.
+   */
+  public void testEmptyHighlights() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+
+    Field body = new Field("body", "test this is.  another sentence this test has.  far away is that planet.", fieldType);
+    doc.add(body);
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    int[] docIDs = new int[]{0};
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertEquals("test this is.  another sentence this test has.  ", snippets[0]);
+
+    ir.close();
+  }
+
+  /**
+   * Not empty but nothing analyzes. Ensures we address null term-vectors.
+   */
+  public void testNothingAnalyzes() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+    doc.add(new Field("body", " ", fieldType));// just a space! (thus not empty)
+    doc.add(newTextField("id", "id", Field.Store.YES));
+    iw.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new Field("body", "something", fieldType));
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
+
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    int[] docIDs = new int[1];
+    docIDs[0] = docID;
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertEquals(" ", snippets[0]);
+
+    ir.close();
+  }
+
+  /**
+   * Make sure highlighter we can customize how emtpy
+   * highlight is returned.
+   */
+  public void testCustomEmptyHighlights() throws Exception {
+    indexAnalyzer.setPositionIncrementGap(10);
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+
+    Field body = new Field("body", "test this is.  another sentence this test has.  far away is that planet.", fieldType);
+    doc.add(body);
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setMaxNoHighlightPassages(0);// don't want any default summary
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    int[] docIDs = new int[]{0};
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertNull(snippets[0]);
+
+    ir.close();
+  }
+
+  /**
+   * Make sure highlighter returns whole text when there
+   * are no hits and BreakIterator is null.
+   */
+  public void testEmptyHighlightsWhole() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+
+    Field body = new Field("body", "test this is.  another sentence this test has.  far away is that planet.", fieldType);
+    doc.add(body);
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected BreakIterator getBreakIterator(String field) {
+        return new WholeBreakIterator();
+      }
+    };
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    int[] docIDs = new int[]{0};
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertEquals("test this is.  another sentence this test has.  far away is that planet.", snippets[0]);
+
+    ir.close();
+  }
+
+  /**
+   * Make sure highlighter is OK with entirely missing
+   * field.
+   */
+  public void testFieldIsMissing() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+
+    Field body = new Field("body", "test this is.  another sentence this test has.  far away is that planet.", fieldType);
+    doc.add(body);
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new TermQuery(new Term("bogus", "highlighting"));
+    int[] docIDs = new int[]{0};
+    String snippets[] = highlighter.highlightFields(new String[]{"bogus"}, query, docIDs, new int[]{2}).get("bogus");
+    assertEquals(1, snippets.length);
+    assertNull(snippets[0]);
+
+    ir.close();
+  }
+
+  public void testFieldIsJustSpace() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+    doc.add(new Field("body", "   ", fieldType));
+    doc.add(newTextField("id", "id", Field.Store.YES));
+    iw.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new Field("body", "something", fieldType));
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
+
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    int[] docIDs = new int[1];
+    docIDs[0] = docID;
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertEquals("   ", snippets[0]);
+
+    ir.close();
+  }
+
+  public void testFieldIsEmptyString() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+    doc.add(new Field("body", "", fieldType));
+    doc.add(newTextField("id", "id", Field.Store.YES));
+    iw.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new Field("body", "something", fieldType));
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
+
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    int[] docIDs = new int[1];
+    docIDs[0] = docID;
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertNull(snippets[0]);
+
+    ir.close();
+  }
+
+  public void testMultipleDocs() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      String content = "the answer is " + i;
+      if ((i & 1) == 0) {
+        content += " some more terms";
+      }
+      doc.add(new Field("body", content, fieldType));
+      doc.add(newStringField("id", "" + i, Field.Store.YES));
+      iw.addDocument(doc);
+
+      if (random().nextInt(10) == 2) {
+        iw.commit();
+      }
+    }
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setCacheFieldValCharsThreshold(random().nextInt(10) * 10);// 0 thru 90 intervals of 10
+    Query query = new TermQuery(new Term("body", "answer"));
+    TopDocs hits = searcher.search(query, numDocs);
+    assertEquals(numDocs, hits.totalHits);
+
+    String snippets[] = highlighter.highlight("body", query, hits);
+    assertEquals(numDocs, snippets.length);
+    for (int hit = 0; hit < numDocs; hit++) {
+      Document doc = searcher.doc(hits.scoreDocs[hit].doc);
+      int id = Integer.parseInt(doc.get("id"));
+      String expected = "the <b>answer</b> is " + id;
+      if ((id & 1) == 0) {
+        expected += " some more terms";
+      }
+      assertEquals(expected, snippets[hit]);
+    }
+
+    ir.close();
+  }
+
+  public void testMultipleSnippetSizes() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Field title = new Field("title", "", UHTestHelper.randomFieldType(random()));
+    Document doc = new Document();
+    doc.add(body);
+    doc.add(title);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    title.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "test")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("title", "test")), BooleanClause.Occur.SHOULD)
+        .build();
+    Map<String, String[]> snippets = highlighter.highlightFields(new String[]{"title", "body"}, query, new int[]{0}, new int[]{1, 2});
+    String titleHighlight = snippets.get("title")[0];
+    String bodyHighlight = snippets.get("body")[0];
+    assertEquals("This is a <b>test</b>. ", titleHighlight);
+    assertEquals("This is a <b>test</b>. Just a <b>test</b> highlighting from postings. ", bodyHighlight);
+    ir.close();
+  }
+
+  public void testEncode() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from <i>postings</i>. Feel free to ignore.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected PassageFormatter getFormatter(String field) {
+        return new DefaultPassageFormatter("<b>", "</b>", "... ", true);
+      }
+    };
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(1, snippets.length);
+    assertEquals("Just&#32;a&#32;test&#32;<b>highlighting</b>&#32;from&#32;&lt;i&gt;postings&lt;&#x2F;i&gt;&#46;&#32;", snippets[0]);
+
+    ir.close();
+  }
+
+  // LUCENE-4906
+  public void testObjectFormatter() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test. Just a test highlighting from postings. Feel free to ignore.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+      @Override
+      protected PassageFormatter getFormatter(String field) {
+        return new PassageFormatter() {
+          PassageFormatter defaultFormatter = new DefaultPassageFormatter();
+
+          @Override
+          public String[] format(Passage passages[], String content) {
+            // Just turns the String snippet into a length 2
+            // array of String
+            return new String[]{"blah blah", defaultFormatter.format(passages, content).toString()};
+          }
+        };
+      }
+    };
+
+    Query query = new TermQuery(new Term("body", "highlighting"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    int[] docIDs = new int[1];
+    docIDs[0] = topDocs.scoreDocs[0].doc;
+    Map<String, Object[]> snippets = highlighter.highlightFieldsAsObjects(new String[]{"body"}, query, docIDs, new int[]{1});
+    Object[] bodySnippets = snippets.get("body");
+    assertEquals(1, bodySnippets.length);
+    assertTrue(Arrays.equals(new String[]{"blah blah", "Just a test <b>highlighting</b> from postings. "}, (String[]) bodySnippets[0]));
+
+    ir.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
new file mode 100644
index 0000000..63f0bb1
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/TestUnifiedHighlighterMTQ.java
@@ -0,0 +1,936 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+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.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TermRangeQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.spans.SpanFirstQuery;
+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.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.junit.After;
+import org.junit.Before;
+
+/**
+ * Some tests that highlight wildcard, fuzzy, etc queries.
+ */
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene3x"})
+@LuceneTestCase.SuppressSysoutChecks(bugUrl = "")//Gradle interferes with this Lucene test rule
+public class TestUnifiedHighlighterMTQ extends LuceneTestCase {
+
+  final FieldType fieldType;
+
+  BaseDirectoryWrapper dir;
+  Analyzer indexAnalyzer;
+
+  @ParametersFactory
+  public static Iterable<Object[]> parameters() {
+    return UHTestHelper.parametersFactoryList();
+  }
+
+  public TestUnifiedHighlighterMTQ(FieldType fieldType) {
+    this.fieldType = fieldType;
+  }
+
+  @Before
+  public void doBefore() throws IOException {
+    dir = newDirectory();
+    indexAnalyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);//whitespace, punctuation, lowercase
+  }
+
+  @After
+  public void doAfter() throws IOException {
+    dir.close();
+  }
+
+  public void testWildcards() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new WildcardQuery(new Term("body", "te*"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // disable MTQ; won't highlight
+    highlighter.setHandleMultiTermQuery(false);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    highlighter.setHandleMultiTermQuery(true);//reset
+
+    // wrong field
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(new WildcardQuery(new Term("bogus", "te*")), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testOnePrefix() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new PrefixQuery(new Term("body", "te"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // wrong field
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(new PrefixQuery(new Term("bogus", "te")), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testOneRegexp() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new RegexpQuery(new Term("body", "te.*"));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // wrong field
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(new RegexpQuery(new Term("bogus", "te.*")), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testOneFuzzy() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new FuzzyQuery(new Term("body", "tets"), 1);
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // with prefix
+    query = new FuzzyQuery(new Term("body", "tets"), 1, 2);
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // wrong field
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(new FuzzyQuery(new Term("bogus", "tets"), 1), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testRanges() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = TermRangeQuery.newStringRange("body", "ta", "tf", true, true);
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // null start
+    query = TermRangeQuery.newStringRange("body", null, "tf", true, true);
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This <b>is</b> <b>a</b> <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> <b>a</b> <b>one</b> <b>sentence</b> <b>document</b>.", snippets[1]);
+
+    // null end
+    query = TermRangeQuery.newStringRange("body", "ta", null, true, true);
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("<b>This</b> is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // exact start inclusive
+    query = TermRangeQuery.newStringRange("body", "test", "tf", true, true);
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // exact end inclusive
+    query = TermRangeQuery.newStringRange("body", "ta", "test", true, true);
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // exact start exclusive
+    BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(TermRangeQuery.newStringRange("body", "test", "tf", false, true), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    // exact end exclusive
+    bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(TermRangeQuery.newStringRange("body", "ta", "test", true, false), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    // wrong field
+    bq = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(TermRangeQuery.newStringRange("bogus", "ta", "tf", true, true), BooleanClause.Occur.SHOULD)
+        .build();
+    topDocs = searcher.search(bq, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testWildcardInBoolean() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    // must not
+    query = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(new WildcardQuery(new Term("bogus", "te*")), BooleanClause.Occur.MUST_NOT)
+        .build();
+    topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testWildcardInFiltered() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term("body", "test")), BooleanClause.Occur.FILTER)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testWildcardInConstantScore() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    ConstantScoreQuery query = new ConstantScoreQuery(new WildcardQuery(new Term("body", "te*")));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testWildcardInDisjunctionMax() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    DisjunctionMaxQuery query = new DisjunctionMaxQuery(
+        Collections.singleton(new WildcardQuery(new Term("body", "te*"))), 0);
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testSpanWildcard() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testSpanOr() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    SpanQuery childQuery = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
+    Query query = new SpanOrQuery(new SpanQuery[]{childQuery});
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testSpanNear() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    SpanQuery childQuery = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
+    Query query = new SpanNearQuery(new SpanQuery[]{childQuery, childQuery}, 0, false);
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testSpanNot() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    SpanQuery include = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
+    SpanQuery exclude = new SpanTermQuery(new Term("body", "bogus"));
+    Query query = new SpanNotQuery(include, exclude);
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  public void testSpanPositionCheck() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("This is a test.");
+    iw.addDocument(doc);
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    SpanQuery childQuery = new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term("body", "te*")));
+    Query query = new SpanFirstQuery(childQuery, 1000000);
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a <b>test</b>.", snippets[0]);
+    assertEquals("<b>Test</b> a one sentence document.", snippets[1]);
+
+    ir.close();
+  }
+
+  /**
+   * Runs a query with two MTQs and confirms the formatter
+   * can tell which query matched which hit.
+   */
+  public void testWhichMTQMatched() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("Test a one sentence document.");
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD)
+        .add(new WildcardQuery(new Term("body", "one")), BooleanClause.Occur.SHOULD)
+        .add(new WildcardQuery(new Term("body", "se*")), BooleanClause.Occur.SHOULD)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, topDocs);
+    assertEquals(1, snippets.length);
+
+    // Default formatter just bolds each hit:
+    assertEquals("<b>Test</b> a <b>one</b> <b>sentence</b> document.", snippets[0]);
+
+    // Now use our own formatter, that also stuffs the
+    // matching term's text into the result:
+    highlighter = new UnifiedHighlighter(searcher, indexAnalyzer) {
+
+      @Override
+      protected PassageFormatter getFormatter(String field) {
+        return new PassageFormatter() {
+
+          @Override
+          public Object format(Passage passages[], String content) {
+            // Copied from DefaultPassageFormatter, but
+            // tweaked to include the matched term:
+            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("... ");
+              }
+              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) {
+                  sb.append(content, pos, start);
+                }
+                if (end > pos) {
+                  sb.append("<b>");
+                  sb.append(content, Math.max(pos, start), end);
+                  sb.append('(');
+                  sb.append(passage.getMatchTerms()[i].utf8ToString());
+                  sb.append(')');
+                  sb.append("</b>");
+                  pos = end;
+                }
+              }
+              // its possible a "term" from the analyzer could span a sentence boundary.
+              sb.append(content, pos, Math.max(pos, passage.endOffset));
+              pos = passage.endOffset;
+            }
+            return sb.toString();
+          }
+        };
+      }
+    };
+
+    assertEquals(1, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, topDocs);
+    assertEquals(1, snippets.length);
+
+    // Default formatter bolds each hit:
+    assertEquals("<b>Test(body:te*)</b> a <b>one(body:one)</b> <b>sentence(body:se*)</b> document.", snippets[0]);
+
+    ir.close();
+  }
+
+
+  //
+  //  All tests below were *not* ported from the PostingsHighlighter; they are new to the U.H.
+  //
+
+  public void testWithMaxLen() throws IOException {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("Alpha Bravo foo foo foo. Foo foo Alpha Bravo");//44 char long, 2 sentences
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    highlighter.setMaxLength(25);//a little past first sentence
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("body", "alpha")), BooleanClause.Occur.MUST)
+        .add(new PrefixQuery(new Term("body", "bra")), BooleanClause.Occur.MUST)
+        .build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    String snippets[] = highlighter.highlight("body", query, topDocs, 2);//ask for 2 but we'll only get 1
+    assertArrayEquals(
+        new String[]{"<b>Alpha</b> <b>Bravo</b> foo foo foo. "}, snippets
+    );
+
+    ir.close();
+  }
+
+  public void testTokenStreamIsClosed() throws IOException {
+    // note: test is a derivative of testWithMaxLen()
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Field body = new Field("body", "", fieldType);
+    Document doc = new Document();
+    doc.add(body);
+
+    body.setStringValue("Alpha Bravo foo foo foo. Foo foo Alpha Bravo");
+    if (random().nextBoolean()) { // sometimes add a 2nd value (maybe matters?)
+      doc.add(new Field("body", "2nd value Alpha Bravo", fieldType));
+    }
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    // use this buggy Analyzer at highlight time
+    Analyzer buggyAnalyzer = new Analyzer() {
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        Tokenizer buggyTokenizer = new Tokenizer() {
+          @Override
+          public boolean incrementToken() throws IOException {
+            throw new IOException("EXPECTED");
+          }
+        };
+        return new TokenStreamComponents(buggyTokenizer);
+      }
+    };
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, buggyAnalyzer);
+    highlighter.setHandleMultiTermQuery(true);
+    if (rarely()) {
+      highlighter.setMaxLength(25);//a little past first sentence
+    }
+
+    boolean hasClauses = false;
+    BooleanQuery.Builder queryBuilder = new BooleanQuery.Builder();
+    if (random().nextBoolean()) {
+      hasClauses = true;
+      queryBuilder.add(new TermQuery(new Term("body", "alpha")), BooleanClause.Occur.MUST);
+    }
+    if (!hasClauses || random().nextBoolean()) {
+      queryBuilder.add(new PrefixQuery(new Term("body", "bra")), BooleanClause.Occur.MUST);
+    }
+    BooleanQuery query = queryBuilder.build();
+    TopDocs topDocs = searcher.search(query, 10, Sort.INDEXORDER);
+    try {
+      String snippets[] = highlighter.highlight("body", query, topDocs, 2);
+      // don't even care what the results are; just want to test exception behavior
+      if (fieldType == UHTestHelper.reanalysisType) {
+        fail("Expecting EXPECTED IOException");
+      }
+    } catch (IOException e) {
+      if (!e.getMessage().equals("EXPECTED")) {
+        throw e;
+      }
+    }
+    ir.close();
+
+    // Now test we can get the tokenStream without it puking due to IllegalStateException for not calling close()
+
+    try (TokenStream ts = buggyAnalyzer.tokenStream("body", "anything")) {
+      ts.reset();// hopefully doesn't throw
+      // don't call incrementToken; we know it's buggy ;-)
+    }
+  }
+
+  /**
+   * Not empty but nothing analyzes. Ensures we address null term-vectors.
+   */
+  public void testNothingAnalyzes() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+    doc.add(new Field("body", " ", fieldType));// just a space! (thus not empty)
+    doc.add(newTextField("id", "id", Field.Store.YES));
+    iw.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new Field("body", "something", fieldType));
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
+
+    Query query = new PrefixQuery(new Term("body", "nonexistent"));
+    int[] docIDs = new int[1];
+    docIDs[0] = docID;
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIDs, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertEquals(" ", snippets[0]);
+
+    ir.close();
+  }
+
+  public void testMultiSegment() throws Exception {
+    // If we incorrectly got the term vector from mis-matched global/leaf doc ID, this test may fail
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+
+    Document doc = new Document();
+    doc.add(new Field("body", "word aberration", fieldType));
+    iw.addDocument(doc);
+
+    iw.commit(); // make segment
+
+    doc = new Document();
+    doc.add(new Field("body", "word absolve", fieldType));
+    iw.addDocument(doc);
+
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    Query query = new PrefixQuery(new Term("body", "ab"));
+    TopDocs topDocs = searcher.search(query, 10);
+
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, topDocs).get("body");
+    Arrays.sort(snippets);
+    assertEquals("[word <b>aberration</b>, word <b>absolve</b>]", Arrays.toString(snippets));
+
+    ir.close();
+  }
+
+  public void testPositionSensitiveWithWildcardDoesNotHighlight() throws Exception {
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, indexAnalyzer);
+    Document doc = new Document();
+    doc.add(new Field("body", "iterate insect ipswitch illinois indirect", fieldType));
+    doc.add(newTextField("id", "id", Field.Store.YES));
+
+    iw.addDocument(doc);
+    IndexReader ir = iw.getReader();
+    iw.close();
+
+    IndexSearcher searcher = newSearcher(ir);
+    UnifiedHighlighter highlighter = new UnifiedHighlighter(searcher, indexAnalyzer);
+    int docID = searcher.search(new TermQuery(new Term("id", "id")), 1).scoreDocs[0].doc;
+
+    PhraseQuery pq = new PhraseQuery.Builder()
+        .add(new Term("body", "consent"))
+        .add(new Term("body", "order"))
+        .build();
+
+    BooleanQuery query = new BooleanQuery.Builder()
+        .add(new WildcardQuery(new Term("body", "enforc*")), BooleanClause.Occur.MUST)
+        .add(pq, BooleanClause.Occur.MUST)
+        .build();
+
+    int[] docIds = new int[]{docID};
+
+    String snippets[] = highlighter.highlightFields(new String[]{"body"}, query, docIds, new int[]{2}).get("body");
+    assertEquals(1, snippets.length);
+    assertEquals("iterate insect ipswitch illinois indirect", snippets[0]);
+    ir.close();
+  }
+
+}


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

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4b679436/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/CambridgeMA.utf8
----------------------------------------------------------------------
diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/CambridgeMA.utf8 b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/CambridgeMA.utf8
new file mode 100644
index 0000000..d60b6fa
--- /dev/null
+++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/CambridgeMA.utf8
@@ -0,0 +1 @@
+{{Distinguish|Cambridge, England}} {{primary sources|date=June 2012}} {{Use mdy dates|date=January 2011}} {{Infobox settlement |official_name = Cambridge, Massachusetts |nickname = |motto = "Boston's Left Bank"<ref>{{cite web|url= http://www.epodunk.com/cgi-bin/genInfo.php?locIndex=2894|title=Profile for Cambridge, Massachusetts, MA|publisher= ePodunk |accessdate= November 1, 2012}}</ref> |image_skyline = CambridgeMACityHall2.jpg |imagesize = 175px |image_caption = Cambridge City Hall |image_seal = |image_flag = |image_map = Cambridge ma highlight.png |mapsize = 250px |map_caption = Location in Middlesex County in Massachusetts |image_map1 = |mapsize1 = |map_caption1 = |coordinates_region = US-MA |subdivision_type = Country |subdivision_name = United States |subdivision_type1 = State |subdivision_name1 = [[Massachusetts]] |subdivision_type2 = [[List of counties in Massachusetts|County]] |subdivision_name2 = [[Middlesex County, Massachusetts|Middlesex]] |established_title = Settled |
 established_date = 1630 |established_title2 = Incorporated |established_date2 = 1636 |established_title3 = |established_date3 = |government_type = [[Council-manager government|Council-City Manager]] |leader_title = Mayor |leader_name = Henrietta Davis |leader_title1 = [[City manager|City Manager]] |leader_name1 = [[Robert W. Healy]] |area_magnitude = |area_total_km2 = 18.47 |area_total_sq_mi = 7.13 |area_land_km2 = 16.65 |area_land_sq_mi = 6.43 |area_water_km2 = 1.81 |area_water_sq_mi = 0.70 |population_as_of = 2010 |population_blank2_title = [[Demonym]] |population_blank2 = [[Cantabrigian]] |settlement_type = City |population_total = 105,162 |population_density_km2 = 6,341.98 |population_density_sq_mi = 16,422.08 |elevation_m = 12 |elevation_ft = 40 |timezone = [[Eastern Time Zone|Eastern]] |utc_offset = -5 |timezone_DST = [[Eastern Time Zone|Eastern]] |utc_offset_DST = -4 |coordinates_display = display=inline,title |latd = 42 |latm = 22 |lats = 25 |latNS = N |longd = 71 |longm = 0
 6 |longs = 38 |longEW = W |website = [http://www.cambridgema.gov/ www.cambridgema.gov] |postal_code_type = ZIP code |postal_code = 02138, 02139, 02140, 02141, 02142 |area_code = [[Area code 617|617]] / [[Area code 857|857]] |blank_name = [[Federal Information Processing Standard|FIPS code]] |blank_info = 25-11000 |blank1_name = [[Geographic Names Information System|GNIS]] feature ID |blank1_info = 0617365 |footnotes = }} '''Cambridge''' is a city in [[Middlesex County, Massachusetts|Middlesex County]], [[Massachusetts]], [[United States]], in the [[Greater Boston]] area. It was named in honor of the [[University of Cambridge]] in [[England]], an important center of the [[Puritan]] theology embraced by the town's founders.<ref>{{cite book|last=Degler|first=Carl Neumann|title=Out of Our Pasts: The Forces That Shaped Modern America|publisher=HarperCollins|location=New York|year=1984|url=http://books.google.com/books?id=NebLe1ueuGQC&pg=PA18&lpg=PA18&dq=cambridge+university+puritans+newt
 owne#v=onepage&q=&f=false|accessdate=September 9, 2009 | isbn=978-0-06-131985-3}}</ref> Cambridge is home to two of the world's most prominent universities, [[Harvard University]] and the [[Massachusetts Institute of Technology]]. According to the [[2010 United States Census]], the city's population was 105,162.<ref name="2010.census.gov">{{cite web|url=http://2010.census.gov/news/releases/operations/cb11-cn104.html |title=Census 2010 News &#124; U.S. Census Bureau Delivers Massachusetts' 2010 Census Population Totals, Including First Look at Race and Hispanic Origin Data for Legislative Redistricting |publisher=2010.census.gov |date=2011-03-22 |accessdate=2012-04-28}}</ref> It is the fifth most populous city in the state, behind [[Boston]], [[Worcester, MA|Worcester]], [[Springfield, MA|Springfield]], and [[Lowell, Massachusetts|Lowell]].<ref name="2010.census.gov"/> Cambridge was one of the two [[county seat]]s of Middlesex County prior to the abolition of county government in 199
 7; [[Lowell, Massachusetts|Lowell]] was the other. ==History== {{See also|Timeline of Cambridge, Massachusetts history}} [[File:Formation of Massachusetts towns.svg|thumb|A map showing the original boundaries of Cambridge]] The site for what would become Cambridge was chosen in December 1630, because it was located safely upriver from Boston Harbor, which made it easily defensible from attacks by enemy ships. Also, the water from the local spring was so good that the local Native Americans believed it had medicinal properties.{{Citation needed|date=November 2009}} [[Thomas Dudley]], his daughter [[Anne Bradstreet]] and her husband Simon were among the first settlers of the town. The first houses were built in the spring of 1631. The settlement was initially referred to as "the newe towne".<ref name=drake>{{cite book|last=Drake|first=Samuel Adams|title=History of Middlesex County, Massachusetts|publisher=Estes and Lauriat|location=Boston|year=1880|volume=1|pages=305\u201316|url=http://b
 ooks.google.com/books?id=QGolOAyd9RMC&pg=PA316&lpg=PA305&dq=newetowne&ct=result#PPA305,M1|accessdate=December 26, 2008}}</ref> Official Massachusetts records show the name capitalized as '''Newe Towne''' by 1632.<ref name=public>{{cite book|title=Report on the Custody and Condition of the Public Records of Parishes|publisher=Massachusetts Secretary of the Commonwealth|url=http://books.google.com/books?id=IyYWAAAAYAAJ&pg=RA1-PA298&lpg=RA1-PA298&dq=%22Ordered+That+Newtowne+shall+henceforward+be+called%22|location=Boston|year=1889|page=298|accessdate=December 24, 2008}}</ref> Located at the first convenient [[Charles River]] crossing west of [[Boston]], Newe Towne was one of a number of towns (including Boston, [[Dorchester, Massachusetts|Dorchester]], [[Watertown, Massachusetts|Watertown]], and [[Weymouth, Massachusetts|Weymouth]]) founded by the 700 original [[Puritan]] colonists of the [[Massachusetts Bay Colony]] under governor [[John Winthrop]]. The original village site is in the
  heart of today's [[Harvard Square]]. The marketplace where farmers brought in crops from surrounding towns to sell survives today as the small park at the corner of John F. Kennedy (J.F.K.) and Winthrop Streets, then at the edge of a salt marsh, since filled. The town included a much larger area than the present city, with various outlying parts becoming independent towns over the years: [[Newton, Massachusetts|Newton (originally Cambridge Village, then Newtown)]] in 1688,<ref>{{cite book |last= Ritter |first= Priscilla R. |coauthors= Thelma Fleishman |title= Newton, Massachusetts 1679\u20131779: A Biographical Directory |year= 1982 |publisher= New England Historic Genealogical Society }}</ref> [[Lexington, Massachusetts|Lexington (Cambridge Farms)]] in 1712, and both [[Arlington, Massachusetts|West Cambridge (originally Menotomy)]] and [[Brighton, Massachusetts|Brighton (Little Cambridge)]] in 1807.<ref>{{cite web |url=http://www.brightonbot.com/history.php |title=A Short History of 
 Allston-Brighton |first=Marchione |last=William P. |author= |authorlink= |coauthors= |date= |month= |year=2011 |work=Brighton-Allston Historical Society |publisher=Brighton Board of Trade |location= |page= |pages= |at= |language= |trans_title= |arxiv= |asin= |bibcode= |doi= |doibroken= |isbn= |issn= |jfm= |jstor= |lccn= |mr= |oclc= |ol= |osti= |pmc = |pmid= |rfc= |ssrn= |zbl= |id= |archiveurl= |archivedate= |deadurl= |accessdate=December 21, 2011 |quote= |ref= |separator= |postscript=}}</ref> Part of West Cambridge joined the new town of [[Belmont, Massachusetts|Belmont]] in 1859, and the rest of West Cambridge was renamed Arlington in 1867; Brighton was annexed by Boston in 1874. In the late 19th century, various schemes for annexing Cambridge itself to the City of Boston were pursued and rejected.<ref>{{cite news |title=ANNEXATION AND ITS FRUITS |author=Staff writer |first= |last= |authorlink= |url=http://query.nytimes.com/gst/abstract.html?res=9901E4DC173BEF34BC4D52DFB766838F669F
 DE |agency= |newspaper=[[The New York Times]] |publisher= |isbn= |issn= |pmid= |pmd= |bibcode= |doi= |date=January 15, 1874, Wednesday |page= 4 |pages= |accessdate=|archiveurl=http://query.nytimes.com/mem/archive-free/pdf?res=9901E4DC173BEF34BC4D52DFB766838F669FDE |archivedate=January 15, 1874 |ref= }}</ref><ref>{{cite news |title=BOSTON'S ANNEXATION SCHEMES.; PROPOSAL TO ABSORB CAMBRIDGE AND OTHER NEAR-BY TOWNS |author=Staff writer |first= |last= |authorlink= |url=http://query.nytimes.com/gst/abstract.html?res=9C05E1DC1F39E233A25754C2A9659C94639ED7CF |agency= |newspaper=[[The New York Times]] |publisher= |isbn= |issn= |pmid= |pmd= |bibcode= |doi= |date=March 26, 1892, Wednesday |page= 11 |pages= |accessdate=August 21, 2010|archiveurl=http://query.nytimes.com/mem/archive-free/pdf?res=9C05E1DC1F39E233A25754C2A9659C94639ED7CF |archivedate=March 27, 1892 |ref= }}</ref> In 1636, [[Harvard College]] was founded by the colony to train [[minister (religion)|ministers]] and the new town was
  chosen for its site by [[Thomas Dudley]]. By 1638, the name "Newe Towne" had "compacted by usage into 'Newtowne'."<ref name=drake /> In May 1638<ref>{{cite book|title=The Cambridge of Eighteen Hundred and Ninety-six|editor=Arthur Gilman, ed.|publisher=Committee on the Memorial Volume|location=Cambridge|year=1896|page=8}}</ref><ref>{{cite web|author=Harvard News Office |url=http://news.harvard.edu/gazette/2002/05.02/02-history.html |title='&#39;Harvard Gazette'&#39; historical calendar giving May 12, 1638 as date of name change; certain other sources say May 2, 1638 or late 1637 |publisher=News.harvard.edu |date=2002-05-02 |accessdate=2012-04-28}}</ref> the name was changed to '''Cambridge''' in honor of the [[University of Cambridge|university]] in [[Cambridge, England]].<ref>{{cite book |last= Hannah Winthrop Chapter, D.A.R. |title= Historic Guide to Cambridge |edition= Second |year= 1907 |publisher= Hannah Winthrop Chapter, D.A.R. |location= Cambridge, Mass. |pages= 20\u201321 |quot
 e= On October&nbsp;15, 1637, the Great and General Court passed a vote that: "The college is ordered to bee at Newetowne." In this same year the name of Newetowne was changed to Cambridge, ("It is ordered that Newetowne shall henceforward be called Cambridge") in honor of the university in Cambridge, England, where many of the early settlers were educated. }}</ref> The first president ([[Henry Dunster]]), the first benefactor ([[John Harvard (clergyman)|John Harvard]]), and the first schoolmaster ([[Nathaniel Eaton]]) of Harvard were all Cambridge University alumni, as was the then ruling (and first) governor of the [[Massachusetts Bay Colony]], John Winthrop. In 1629, Winthrop had led the signing of the founding document of the city of Boston, which was known as the [[Cambridge Agreement]], after the university.<ref>{{cite web|url=http://www.winthropsociety.org/doc_cambr.php|publisher=The Winthrop Society|title=Descendants of the Great Migration|accessdate=September 8, 2008}}</ref>
  It was Governor Thomas Dudley who, in 1650, signed the charter creating the corporation which still governs Harvard College.<ref>{{cite web|url=http://hul.harvard.edu/huarc/charter.html |title=Harvard Charter of 1650, Harvard University Archives, Harvard University, harvard.edu |publisher=Hul.harvard.edu |date= |accessdate=2012-04-28}}</ref><ref>{{cite book |last1= |first1= |authorlink1= |editor1-first= |editor1-last= |editor1-link= |others= |title=Constitution of the Commonwealth of Massachusetts|url=http://www.mass.gov/legis/const.htm |accessdate=December 13, 2009 |edition= |series= |volume= |date=September 1, 1779 |publisher=The General Court of Massachusetts |location= |isbn= |oclc= |doi= |page= |pages=|chapter=Chapter V: The University at Cambridge, and encouragement of literature, etc. |chapterurl= |ref= |bibcode= }}</ref> [[Image:Washington taking command of the American Army at Cambridge, 1775 - NARA - 532874.tif|thumb|right|George Washington in Cambridge, 1775]] Cambridge 
 grew slowly as an agricultural village eight miles (13&nbsp;km) by road from Boston, the capital of the colony. By the [[American Revolution]], most residents lived near the [[Cambridge Common|Common]] and Harvard College, with farms and estates comprising most of the town. Most of the inhabitants were descendants of the original Puritan colonists, but there was also a small elite of [[Anglicans|Anglican]] "worthies" who were not involved in village life, who made their livings from estates, investments, and trade, and lived in mansions along "the Road to Watertown" (today's [[Brattle Street (Cambridge, Massachusetts)|Brattle Street]], still known as [[Tory Row]]). In 1775, [[George Washington]] came up from [[Virginia]] to take command of fledgling volunteer American soldiers camped on the [[Cambridge Common]]\u2014today called the birthplace of the [[U.S. Army]]. (The name of today's nearby Sheraton Commander Hotel refers to that event.) Most of the Tory estates were confiscated afte
 r the Revolution. On January 24, 1776, [[Henry Knox]] arrived with artillery captured from [[Fort Ticonderoga]], which enabled Washington to drive the British army out of Boston. [[File:Cambridge 1873 WardMap.jpg|thumb|300px|left|A map of Cambridge from 1873]] Between 1790 and 1840, Cambridge began to grow rapidly, with the construction of the [[West Boston Bridge]] in 1792, that connected Cambridge directly to Boston, making it no longer necessary to travel eight miles (13&nbsp;km) through the [[Boston Neck]], [[Roxbury, Massachusetts|Roxbury]], and [[Brookline, Massachusetts|Brookline]] to cross the [[Charles River]]. A second bridge, the Canal Bridge, opened in 1809 alongside the new [[Middlesex Canal]]. The new bridges and roads made what were formerly estates and [[marsh]]land into prime industrial and residential districts. In the mid-19th century, Cambridge was the center of a literary revolution when it gave the country a new identity through poetry and literature. Cambridge
  was home to the famous Fireside Poets\u2014so called because their poems would often be read aloud by families in front of their evening fires. In their day, the [[Fireside Poets]]\u2014[[Henry Wadsworth Longfellow]], [[James Russell Lowell]], and [[Oliver Wendell Holmes, Sr.|Oliver Wendell Holmes]]\u2014were as popular and influential as rock stars are today.{{Citation needed|date=November 2009}} Soon after, [[Toll road|turnpikes]] were built: the [[Cambridge and Concord Turnpike]] (today's Broadway and Concord Ave.), the [[Middlesex Turnpike (Massachusetts)|Middlesex Turnpike]] (Hampshire St. and [[Massachusetts Avenue (Boston)|Massachusetts Ave.]] northwest of [[Porter Square]]), and what are today's Cambridge, Main, and Harvard Streets were roads to connect various areas of Cambridge to the bridges. In addition, railroads crisscrossed the town during the same era, leading to the development of Porter Square as well as the creation of neighboring town [[Somerville, Massachusetts|Somervil
 le]] from the formerly rural parts of [[Charlestown, Massachusetts|Charlestown]]. [[File:Middlesex Canal (Massachusetts) map, 1852.jpg|thumb|1852 Map of Boston area showing Cambridge and rail lines.]] Cambridge was incorporated as a city in 1846. This was despite noticeable tensions between East Cambridge, Cambridgeport, and Old Cambridge that stemmed from differences in in each area's culture, sources of income, and the national origins of the residents.<ref>Cambridge Considered: A Very Brief History of Cambridge, 1800-1900, Part I. http://cambridgeconsidered.blogspot.com/2011/01/very-brief-history-of-cambridge-1800.html</ref> The city's commercial center began to shift from Harvard Square to Central Square, which became the downtown of the city around this time. Between 1850 and 1900, Cambridge took on much of its present character\u2014[[streetcar suburb]]an development along the turnpikes, with working-class and industrial neighborhoods focused on East Cambridge, comfortable middle
 -class housing being built on old estates in Cambridgeport and Mid-Cambridge, and upper-class enclaves near Harvard University and on the minor hills of the city. The coming of the railroad to North Cambridge and Northwest Cambridge then led to three major changes in the city: the development of massive brickyards and brickworks between Massachusetts Ave., Concord Ave. and [[Alewife Brook]]; the ice-cutting industry launched by [[Frederic Tudor]] on [[Fresh Pond, Cambridge, Massachusetts|Fresh Pond]]; and the carving up of the last estates into residential subdivisions to provide housing to the thousands of immigrants that arrived to work in the new industries. For many years, the city's largest employer was the [[New England Glass Company]], founded in 1818. By the middle of the 19th century it was the largest and most modern glassworks in the world. In 1888, all production was moved, by [[Edward Libbey|Edward Drummond Libbey]], to [[Toledo, Ohio]], where it continues today under t
 he name Owens Illinois. Flint glassware with heavy lead content, produced by that company, is prized by antique glass collectors. There is none on public display in Cambridge, but there is a large collection in the [[Toledo Museum of Art]]. Among the largest businesses located in Cambridge was the firm of [[Carter's Ink Company]], whose neon sign long adorned the [[Charles River]] and which was for many years the largest manufacturer of ink in the world. By 1920, Cambridge was one of the main industrial cities of [[New England]], with nearly 120,000 residents. As industry in New England began to decline during the [[Great Depression]] and after World War II, Cambridge lost much of its industrial base. It also began the transition to being an intellectual, rather than an industrial, center. Harvard University had always been important in the city (both as a landowner and as an institution), but it began to play a more dominant role in the city's life and culture. Also, the move of th
 e [[Massachusetts Institute of Technology]] from Boston in 1916 ensured Cambridge's status as an intellectual center of the United States. After the 1950s, the city's population began to decline slowly, as families tended to be replaced by single people and young couples. The 1980s brought a wave of high-technology startups, creating software such as [[Visicalc]] and [[Lotus 1-2-3]], and advanced computers, but many of these companies fell into decline with the fall of the minicomputer and [[DOS]]-based systems. However, the city continues to be home to many startups as well as a thriving biotech industry. By the end of the 20th century, Cambridge had one of the most expensive housing markets in the Northeastern United States. While maintaining much diversity in class, race, and age, it became harder and harder for those who grew up in the city to be able to afford to stay. The end of [[rent control]] in 1994 prompted many Cambridge renters to move to housing that was more affordabl
 e, in Somerville and other communities. In 2005, a reassessment of residential property values resulted in a disproportionate number of houses owned by non-affluent people jumping in value relative to other houses, with hundreds having their property tax increased by over 100%; this forced many homeowners in Cambridge to move elsewhere.<ref>Cambridge Chronicle, October 6, 13, 20, 27, 2005</ref> As of 2012, Cambridge's mix of amenities and proximity to Boston has kept housing prices relatively stable. ==Geography== [[File:Charles River Cambridge USA.jpg|thumb|upright|A view from Boston of Harvard's [[Weld Boathouse]] and Cambridge in winter. The [[Charles River]] is in the foreground.]] According to the [[United States Census Bureau]], Cambridge has a total area of {{convert|7.1|sqmi|km2}}, of which {{convert|6.4|sqmi|km2}} of it is land and {{convert|0.7|sqmi|km2}} of it (9.82%) is water. ===Adjacent municipalities=== Cambridge is located in eastern Massachusetts, bordered by: *the 
 city of [[Boston]] to the south (across the [[Charles River]]) and east *the city of [[Somerville, Massachusetts|Somerville]] to the north *the town of [[Arlington, Massachusetts|Arlington]] to the northwest *the town of [[Belmont, Massachusetts|Belmont]] and *the city of [[Watertown, Massachusetts|Watertown]] to the west The border between Cambridge and the neighboring city of [[Somerville, Massachusetts|Somerville]] passes through densely populated neighborhoods which are connected by the [[Red Line (MBTA)|MBTA Red Line]]. Some of the main squares, [[Inman Square|Inman]], [[Porter Square|Porter]], and to a lesser extent, [[Harvard Square|Harvard]], are very close to the city line, as are Somerville's [[Union Square (Somerville)|Union]] and [[Davis Square]]s. ===Neighborhoods=== ====Squares==== [[File:Centralsquarecambridgemass.jpg|thumb|[[Central Square (Cambridge)|Central Square]]]] [[File:Harvard square 2009j.JPG|thumb|[[Harvard Square]]]] [[File:Cambridge MA Inman Square.jpg|th
 umb|[[Inman Square]]]] Cambridge has been called the "City of Squares" by some,<ref>{{cite web|author=No Writer Attributed |url=http://www.thecrimson.com/article/1969/9/18/cambridge-a-city-of-squares-pcambridge/ |title="Cambridge: A City of Squares" Harvard Crimson, Sept. 18, 1969 |publisher=Thecrimson.com |date=1969-09-18 |accessdate=2012-04-28}}</ref><ref>{{cite web|url=http://www.travelwritersmagazine.com/RonBernthal/Cambridge.html |title=Cambridge Journal: Massachusetts City No Longer in Boston's Shadow |publisher=Travelwritersmagazine.com |date= |accessdate=2012-04-28}}</ref> as most of its commercial districts are major street intersections known as [[Town square|squares]]. Each of the squares acts as a neighborhood center. These include: * [[Kendall Square]], formed by the junction of Broadway, Main Street, and Third Street, is also known as '''Technology Square''', a name shared with an office and laboratory building cluster in the neighborhood. Just over the [[Longfellow Br
 idge]] from Boston, at the eastern end of the [[Massachusetts Institute of Technology|MIT]] campus, it is served by the [[Kendall (MBTA station)|Kendall/MIT]] station on the [[Massachusetts Bay Transportation Authority|MBTA]] [[Red Line (MBTA)|Red Line]] subway. Most of Cambridge's large office towers are located here, giving the area somewhat of an office park feel. A flourishing [[biotech]] industry has grown up around this area. The "One Kendall Square" complex is nearby, but\u2014confusingly\u2014not actually in Kendall Square. Also, the "Cambridge Center" office complex is located here, and not at the actual center of Cambridge. * [[Central Square (Cambridge)|Central Square]], formed by the junction of Massachusetts Avenue, Prospect Street, and Western Avenue, is well known for its wide variety of ethnic restaurants. As recently as the late 1990s it was rather run-down; it underwent a controversial [[gentrification]] in recent years (in conjunction with the development of the nearby 
 [[University Park at MIT]]), and continues to grow more expensive. It is served by the [[Central (MBTA station)|Central Station]] stop on the MBTA Red Line subway. '''Lafayette Square''', formed by the junction of Massachusetts Avenue, Columbia Street, Sidney Street, and Main Street, is considered part of the Central Square area. [[Cambridgeport]] is south of Central Square along Magazine Street and Brookline Street. * [[Harvard Square]], formed by the junction of Massachusetts Avenue, Brattle Street, and JFK Street. This is the primary site of [[Harvard University]], and is a major Cambridge shopping area. It is served by a [[Harvard (MBTA station)|Red Line station]]. Harvard Square was originally the northwestern terminus of the Red Line and a major transfer point to streetcars that also operated in a short [[Harvard Bus Tunnel|tunnel]]\u2014which is still a major bus terminal, although the area under the Square was reconfigured dramatically in the 1980s when the Red Line was extende
 d. The Harvard Square area includes '''Brattle Square''' and '''Eliot Square'''. A short distance away from the square lies the [[Cambridge Common]], while the neighborhood north of Harvard and east of Massachusetts Avenue is known as Agassiz in honor of the famed scientist [[Louis Agassiz]]. * [[Porter Square]], about a mile north on Massachusetts Avenue from Harvard Square, is formed by the junction of Massachusetts and Somerville Avenues, and includes part of the city of [[Somerville, Massachusetts|Somerville]]. It is served by the [[Porter (MBTA station)|Porter Square Station]], a complex housing a [[Red Line (MBTA)|Red Line]] stop and a [[Fitchburg Line]] [[MBTA commuter rail|commuter rail]] stop. [[Lesley University]]'s University Hall and Porter campus are located at Porter Square. * [[Inman Square]], at the junction of Cambridge and Hampshire streets in Mid-Cambridge. Inman Square is home to many diverse restaurants, bars, music venues and boutiques. The funky street scene s
 till holds some urban flair, but was dressed up recently with Victorian streetlights, benches and bus stops. A new community park was installed and is a favorite place to enjoy some takeout food from the nearby restaurants and ice cream parlor. * [[Lechmere Square]], at the junction of Cambridge and First streets, adjacent to the CambridgeSide Galleria shopping mall. Perhaps best known as the northern terminus of the [[Massachusetts Bay Transportation Authority|MBTA]] [[Green Line (MBTA)|Green Line]] subway, at [[Lechmere (MBTA station)|Lechmere Station]]. ====Other neighborhoods==== The residential neighborhoods ([http://www.cambridgema.gov/CPD/publications/neighborhoods.cfm map]) in Cambridge border, but are not defined by the squares. These include: * [[East Cambridge, Massachusetts|East Cambridge]] (Area 1) is bordered on the north by the [[Somerville, Massachusetts|Somerville]] border, on the east by the Charles River, on the south by Broadway and Main Street, and on the west b
 y the [[Grand Junction Railroad]] tracks. It includes the [[NorthPoint (Cambridge, Massachusetts)|NorthPoint]] development. * [[Massachusetts Institute of Technology|MIT]] Campus ([[MIT Campus (Area 2), Cambridge|Area 2]]) is bordered on the north by Broadway, on the south and east by the Charles River, and on the west by the Grand Junction Railroad tracks. * [[Wellington-Harrington]] (Area 3) is bordered on the north by the [[Somerville, Massachusetts|Somerville]] border, on the south and west by Hampshire Street, and on the east by the Grand Junction Railroad tracks. Referred to as "Mid-Block".{{clarify|What is? By whom? A full sentence would help.|date=September 2011}} * [[Area 4, Cambridge|Area 4]] is bordered on the north by Hampshire Street, on the south by Massachusetts Avenue, on the west by Prospect Street, and on the east by the Grand Junction Railroad tracks. Residents of Area 4 often refer to their neighborhood simply as "The Port", and refer to the area of Cambridgeport
  and Riverside as "The Coast". * [[Cambridgeport]] (Area 5) is bordered on the north by Massachusetts Avenue, on the south by the Charles River, on the west by River Street, and on the east by the Grand Junction Railroad tracks. * [[Mid-Cambridge]] (Area 6) is bordered on the north by Kirkland and Hampshire Streets and the [[Somerville, Massachusetts|Somerville]] border, on the south by Massachusetts Avenue, on the west by Peabody Street, and on the east by Prospect Street. * [[Riverside, Cambridge|Riverside]] (Area 7), an area sometimes referred to as "The Coast," is bordered on the north by Massachusetts Avenue, on the south by the Charles River, on the west by JFK Street, and on the east by River Street. * [[Agassiz, Cambridge, Massachusetts|Agassiz (Harvard North)]] (Area 8) is bordered on the north by the [[Somerville, Massachusetts|Somerville]] border, on the south and east by Kirkland Street, and on the west by Massachusetts Avenue. * [[Peabody, Cambridge, Massachusetts|Peabo
 dy]] (Area 9) is bordered on the north by railroad tracks, on the south by Concord Avenue, on the west by railroad tracks, and on the east by Massachusetts Avenue. The Avon Hill sub-neighborhood consists of the higher elevations bounded by Upland Road, Raymond Street, Linnaean Street and Massachusetts Avenue. * Brattle area/[[West Cambridge (neighborhood)|West Cambridge]] (Area 10) is bordered on the north by Concord Avenue and Garden Street, on the south by the Charles River and the [[Watertown, Massachusetts|Watertown]] border, on the west by Fresh Pond and the Collins Branch Library, and on the east by JFK Street. It includes the sub-neighborhoods of Brattle Street (formerly known as [[Tory Row]]) and Huron Village. * [[North Cambridge, Massachusetts|North Cambridge]] (Area 11) is bordered on the north by the [[Arlington, Massachusetts|Arlington]] and [[Somerville, Massachusetts|Somerville]] borders, on the south by railroad tracks, on the west by the [[Belmont, Massachusetts|Bel
 mont]] border, and on the east by the [[Somerville, Massachusetts|Somerville]] border. * [[Cambridge Highlands]] (Area 12) is bordered on the north and east by railroad tracks, on the south by Fresh Pond, and on the west by the [[Belmont, Massachusetts|Belmont]] border. * [[Strawberry Hill, Cambridge|Strawberry Hill]] (Area 13) is bordered on the north by Fresh Pond, on the south by the [[Watertown, Massachusetts|Watertown]] border, on the west by the [[Belmont, Massachusetts|Belmont]] border, and on the east by railroad tracks. ===Parks and outdoors=== [[File:Alewife Brook Reservation.jpg|thumb|Alewife Brook Reservation]] Consisting largely of densely built residential space, Cambridge lacks significant tracts of public parkland. This is partly compensated for, however, by the presence of easily accessible open space on the university campuses, including [[Harvard Yard]] and MIT's Great Lawn, as well as the considerable open space of [[Mount Auburn Cemetery]]. At the western edge o
 f Cambridge, the cemetery is well known as the first garden cemetery, for its distinguished inhabitants, for its superb landscaping (the oldest planned landscape in the country), and as a first-rate [[arboretum]]. Although known as a Cambridge landmark, much of the cemetery lies within the bounds of Watertown.<ref>http://www2.cambridgema.gov/CityOfCambridge_Content/documents/CambridgeStreetMap18x24_032007.pdf</ref> It is also a significant [[Important Bird Area]] (IBA) in the Greater Boston area. Public parkland includes the esplanade along the Charles River, which mirrors its [[Charles River Esplanade|Boston counterpart]], [[Cambridge Common]], a busy and historic public park immediately adjacent to the Harvard campus, and the [[Alewife Brook Reservation]] and [[Fresh Pond, Cambridge, Massachusetts|Fresh Pond]] in the western part of the city. ==Demographics== {{Historical populations | type=USA | align=right | 1790|2115 | 1800|2453 | 1810|2323 | 1820|3295 | 1830|6072 | 1840|8409 |
  1850|15215 | 1860|26060 | 1870|39634 | 1880|52669 | 1890|70028 | 1900|91886 | 1910|104839 | 1920|109694 | 1930|113643 | 1940|110879 | 1950|120740 | 1960|107716 | 1970|100361 | 1980|95322 | 1990|95802 | 2000|101355 | 2010|105162 | footnote= {{Historical populations/Massachusetts municipalities references}}<ref name="1950_Census_Urban_populations_since_1790">{{cite journal | title=1950 Census of Population | volume=1: Number of Inhabitants | at=Section 6, Pages 21-7 through 21-09, Massachusetts Table 4. Population of Urban Places of 10,000 or more from Earliest Census to 1920 | publisher=Bureau of the Census | accessdate=July 12, 2011 | year=1952 | url=http://www2.census.gov/prod2/decennial/documents/23761117v1ch06.pdf}}</ref> }} As of the census{{GR|2}} of 2010, there were 105,162 people, 44,032 households, and 17,420 families residing in the city. The population density was 16,422.08 people per square mile (6,341.98/km�), making Cambridge the fifth most densely populated city in t
 he US<ref name=CountyCityDataBook>County and City Data Book: 2000. Washington, DC: US Department of Commerce, Bureau of the Census. Table C-1.</ref> and the second most densely populated city in [[Massachusetts]] behind neighboring [[Somerville, Massachusetts|Somerville]].<ref>[http://www.boston.com/realestate/news/articles/2008/07/13/highest_population_density/ Highest Population Density, The Boston Globe]</ref> There were 47,291 housing units at an average density of 7,354.7 per square mile (2,840.3/km�). The racial makeup of the city was 66.60% [[White (U.S. Census)|White]], 11.70% [[Black (people)|Black]] or [[Race (United States Census)|African American]], 0.20% [[Native American (U.S. Census)|Native American]], 15.10% [[Asian (U.S. Census)|Asian]], 0.01% [[Pacific Islander (U.S. Census)|Pacific Islander]], 2.10% from [[Race (United States Census)|other races]], and 4.30% from two or more races. 7.60% of the population were [[Hispanics in the United States|Hispanic]] or [[Lati
 no (U.S. Census)|Latino]] of any race. [[Non-Hispanic Whites]] were 62.1% of the population in 2010,<ref>{{cite web |url=http://quickfacts.census.gov/qfd/states/25/2511000.html |title=Cambridge (city), Massachusetts |work=State & County QuickFacts |publisher=U.S. Census Bureau}}</ref> down from 89.7% in 1970.<ref>{{cite web|title=Massachusetts - Race and Hispanic Origin for Selected Cities and Other Places: Earliest Census to 1990|publisher=U.S. Census Bureau|url=http://www.census.gov/population/www/documentation/twps0076/twps0076.html}}</ref> This rather closely parallels the average [[racial demographics of the United States]] as a whole, although Cambridge has significantly more Asians than the average, and fewer Hispanics and Caucasians. 11.0% were of [[irish people|Irish]], 7.2% English, 6.9% [[italians|Italian]], 5.5% [[West Indian]] and 5.3% [[germans|German]] ancestry according to [[Census 2000]]. 69.4% spoke English, 6.9% Spanish, 3.2% [[Standard Mandarin|Chinese]] or [[Sta
 ndard Mandarin|Mandarin]], 3.0% [[portuguese language|Portuguese]], 2.9% [[French-based creole languages|French Creole]], 2.3% French, 1.5% [[korean language|Korean]], and 1.0% [[italian language|Italian]] as their first language. There were 44,032 households out of which 16.9% had children under the age of 18 living with them, 28.9% were married couples living together, 8.4% had a female householder with no husband present, and 60.4% were non-families. 40.7% of all households were made up of individuals and 9.6% had someone living alone who was 65 years of age or older. The average household size was 2.00 and the average family size was 2.76. In the city the population was spread out with 13.3% under the age of 18, 21.2% from 18 to 24, 38.6% from 25 to 44, 17.8% from 45 to 64, and 9.2% who were 65 years of age or older. The median age was 30.5 years. For every 100 females, there were 96.1 males. For every 100 females age 18 and over, there were 94.7 males. The median income for a h
 ousehold in the city was $47,979, and the median income for a family was $59,423 (these figures had risen to $58,457 and $79,533 respectively {{as of|2007|alt=as of a 2007 estimate}}<ref>{{cite web|url=http://factfinder.census.gov/servlet/ACSSAFFFacts?_event=Search&geo_id=16000US2418750&_geoContext=01000US%7C04000US24%7C16000US2418750&_street=&_county=cambridge&_cityTown=cambridge&_state=04000US25&_zip=&_lang=en&_sse=on&ActiveGeoDiv=geoSelect&_useEV=&pctxt=fph&pgsl=160&_submenuId=factsheet_1&ds_name=ACS_2007_3YR_SAFF&_ci_nbr=null&qr_name=null&reg=null%3Anull&_keyword=&_industry= |title=U.S. Census, 2000 |publisher=Factfinder.census.gov |date= |accessdate=2012-04-28}}</ref>). Males had a median income of $43,825 versus $38,489 for females. The per capita income for the city was $31,156. About 8.7% of families and 12.9% of the population were below the poverty line, including 15.1% of those under age 18 and 12.9% of those age 65 or over. Cambridge was ranked as one of the most liberal
  cities in America.<ref>{{cite web|author=Aug 16, 2005 12:00 AM |url=http://www.govpro.com/News/Article/31439/ |title=Study Ranks America\u2019s Most Liberal and Conservative Cities |publisher=Govpro.com |date=2005-08-16 |accessdate=2012-04-28}}</ref> Locals living in and near the city jokingly refer to it as "The People's Republic of Cambridge."<ref>[http://www.universalhub.com/glossary/peoples_republic_the.html Wicked Good Guide to Boston English] Accessed February 2, 2009</ref> For 2012, the residential property tax rate in Cambridge is $8.48 per $1,000.<ref>{{cite web|url=http://www.cambridgema.gov/finance/propertytaxinformation/fy12propertytaxinformation.aspx |title=FY12 Property Tax Information - City of Cambridge, Massachusetts |publisher=Cambridgema.gov |date= |accessdate=2012-04-28}}</ref> Cambridge enjoys the highest possible [[bond credit rating]], AAA, with all three Wall Street rating agencies.<ref>http://www.cambridgema.gov/CityOfCambridge_Content/documents/Understanding_
 Your_Taxes_2007.pdf</ref> Cambridge is noted for its diverse population, both racially and economically. Residents, known as ''Cantabrigians'', include affluent [[MIT]] and Harvard professors. The first legal applications in America for same-sex marriage licenses were issued at Cambridge's City Hall.<ref>{{cite web|url=http://www.boston.com/news/local/articles/2004/05/17/free_to_marry/ |title=Free to Marry |work=[[The Boston Globe]] |date=2004-05-17 |accessdate=2012-07-18}}</ref> Cambridge is also the birthplace of [[Thailand|Thai]] king [[Bhumibol Adulyadej|Bhumibol Adulyadej (Rama IX)]], who is the world's longest reigning monarch at age 82 (2010), as well as the longest reigning monarch in Thai history. He is also the first king of a foreign country to be born in the United States. ==Government== ===Federal and state representation=== {| class=wikitable ! colspan = 6 | Voter registration and party enrollment {{as of|lc=y|df=US|2008|10|15}}<ref>{{cite web|title = 2008 State Party 
 Election Party Enrollment Statistics | publisher = Massachusetts Elections Division | format = PDF | accessdate = July 7, 2010 | url = http://www.sec.state.ma.us/ele/elepdf/st_county_town_enroll_breakdown_08.pdf}}</ref> |- ! colspan = 2 | Party ! Number of voters ! Percentage {{American politics/party colors/Democratic/row}} | [[Democratic Party (United States)|Democratic]] | style="text-align:center;"| 37,822 | style="text-align:center;"| 58.43% {{American politics/party colors/Republican/row}} | [[Republican Party (United States)|Republican]] | style="text-align:center;"| 3,280 | style="text-align:center;"| 5.07% {{American politics/party colors/Independent/row}} | Unaffiliated | style="text-align:center;"| 22,935 | style="text-align:center;"| 35.43% {{American politics/party colors/Libertarian/row}} | Minor Parties | style="text-align:center;"| 690 | style="text-align:center;"| 1.07% |- ! colspan = 2 | Total ! style="text-align:center;"| 64,727 ! style="text-align:center;"| 100% 
 |} Cambridge is part of [[Massachusetts's 8th congressional district]], represented by Democrat [[Mike Capuano]], elected in 1998. The state's senior member of the [[United States Senate]] is Democrat [[John Kerry]], elected in 1984. The state's junior member is Republican [[Scott Brown]], [[United States Senate special election in Massachusetts, 2010|elected in 2010]] to fill the vacancy caused by the death of long-time Democratic Senator [[Ted Kennedy]]. The Governor of Massachusetts is Democrat [[Deval Patrick]], elected in 2006 and re-elected in 2010. On the state level, Cambridge is represented in six districts in the [[Massachusetts House of Representatives]]: the 24th Middlesex (which includes parts of Belmont and Arlington), the 25th and 26th Middlesex (the latter which includes a portion of Somerville), the 29th Middlesex (which includes a small part of Watertown), and the Eighth and Ninth Suffolk (both including parts of the City of Boston). The city is represented in the 
 [[Massachusetts Senate]] as a part of the "First Suffolk and Middlesex" district (this contains parts of Boston, Revere and Winthrop each in Suffolk County); the "Middlesex, Suffolk and Essex" district, which includes Everett and Somerville, with Boston, Chelsea, and Revere of Suffolk, and Saugus in Essex; and the "Second Suffolk and Middlesex" district, containing parts of the City of Boston in Suffolk county, and Cambridge, Belmont and Watertown in Middlesex county.<ref>{{cite web|url=http://www.malegislature.gov/ |title=Index of Legislative Representation by City and Town, from |publisher=Mass.gov |date= |accessdate=2012-04-28}}</ref> In addition to the [[Cambridge Police Department (Massachusetts)|Cambridge Police Department]], the city is patrolled by the Fifth (Brighton) Barracks of Troop H of the [[Massachusetts State Police]].<ref>[http://www.mass.gov/?pageID=eopsterminal&L=5&L0=Home&L1=Law+Enforcement+%26+Criminal+Justice&L2=Law+Enforcement&L3=State+Police+Troops&L4=Troop+H
 &sid=Eeops&b=terminalcontent&f=msp_divisions_field_services_troops_troop_h_msp_field_troop_h_station_h5&csid=Eeops Station H-5, SP Brighton]{{dead link|date=April 2012}}</ref> Due, however, to close proximity, the city also practices functional cooperation with the Fourth (Boston) Barracks of Troop H, as well.<ref>[http://www.mass.gov/?pageID=eopsterminal&L=5&L0=Home&L1=Law+Enforcement+%26+Criminal+Justice&L2=Law+Enforcement&L3=State+Police+Troops&L4=Troop+H&sid=Eeops&b=terminalcontent&f=msp_divisions_field_services_troops_troop_h_msp_field_troop_h_station_h4&csid=Eeops Station H-4, SP Boston]{{dead link|date=April 2012}}</ref> ===City government=== [[File:CambridgeMACityHall1.jpg|thumb|right|[[Cambridge, Massachusetts City Hall|Cambridge City Hall]] in the 1980s]] Cambridge has a city government led by a [[List of mayors of Cambridge, Massachusetts|Mayor]] and nine-member City Council. There is also a six-member School Committee which functions alongside the Superintendent of publi
 c schools. The councilors and school committee members are elected every two years using the [[single transferable vote]] (STV) system.<ref>{{cite web|url=http://www.cambridgema.gov/election/Proportional_Representation.cfm |title=Proportional Representation Voting in Cambridge |publisher=Cambridgema.gov |date= |accessdate=2012-04-28}}</ref> Once a laborious process that took several days to complete by hand, ballot sorting and calculations to determine the outcome of elections are now quickly performed by computer, after the ballots have been [[Optical scan voting system|optically scanned]]. The mayor is elected by the city councilors from amongst themselves, and serves as the chair of City Council meetings. The mayor also sits on the School Committee. However, the Mayor is not the Chief Executive of the City. Rather, the City Manager, who is appointed by the City Council, serves in that capacity. Under the City's Plan E form of government the city council does not have the power to
  appoint or remove city officials who are under direction of the city manager. The city council and its individual members are also forbidden from giving orders to any subordinate of the city manager.<ref>http://www.cambridgema.gov/CityOfCambridge_Content/documents/planE.pdf</ref> [[Robert W. Healy]] is the City Manager; he has served in the position since 1981. In recent history, the media has highlighted the salary of the City Manager as being one of the highest in the State of Massachusetts.<ref>{{cite news |title=Cambridge city manager's salary almost as much as Obama's pay |url=http://www.wickedlocal.com/cambridge/features/x1837730973/Cambridge-city-managers-salary-almost-as-much-as-Obamas |agency= |newspaper=Wicked Local: Cambridge |publisher= |date=August 11, 2011 |accessdate=December 30, 2011 |quote= |archiveurl= |archivedate= |deadurl= |ref=}}</ref> The city council consists of:<ref>{{cite web|url=http://www.cambridgema.gov/ccouncil/citycouncilmembers.aspx |title=City of Ca
 mbridge \u2013 City Council Members |publisher=Cambridgema.gov |date= |accessdate=2012-04-28}}</ref>{{Refbegin|3}} *[[Leland Cheung]] (Jan. 2010\u2013present) *Henrietta Davis (Jan. 1996\u2013present)* *Marjorie C. Decker (Jan. 2000\u2013present)<ref>{{cite web |url= http://www.wickedlocal.com/cambridge/news/x738245499/Marjorie-Decker-announces-she-will-run-for-Alice-Wolfs-Cambridge-State-Representative-seat |title= Marjorie Decker announces she will run for Alice Wolf's Cambridge State Representative seat |date= 22 March 2012 |work= Wicked Local Cambridge |publisher= GateHouse Media, Inc. |accessdate= 4 April 2012 }}</ref> *Craig A. Kelley (Jan. 2006\u2013present) *David Maher (Jan. 2000-Jan. 2006, Sept. 2007\u2013present<ref>{{cite web|author=By ewelin, on September 5th, 2007 |url=http://www.cambridgehighlands.com/2007/09/david-p-maher-elected-to-fill-michael-sullivans-vacated-city-council-seat |title=David P. Maher Elected to fill Michael Sullivan\u2019s Vacated City Council Seat \u2022 Cambridge Highla
 nds Neighborhood Association |publisher=Cambridgehighlands.com |date=2007-09-05 |accessdate=2012-04-28}}</ref>)** *[[Kenneth Reeves]] (Jan. 1990\u2013present)** *[[E. Denise Simmons]] (Jan. 2002\u2013present)** *[[Timothy J. Toomey, Jr.]] (Jan. 1990\u2013present) *Minka vanBeuzekom (Jan. 2012\u2013present){{Refend}} ''* = Current Mayor''<br> ''** = former Mayor'' ===Fire Department=== The city of Cambridge is protected full-time by the 274 professional firefighters of the Cambridge Fire Department. The current Chief of Department is Gerald R. Reardon. The Cambridge Fire Department operates out of eight fire stations, located throughout the city, under the command of two divisions. The CFD also maintains and operates a front-line fire apparatus fleet of eight engines, four ladders, two Non-Transport Paramedic EMS units, a Haz-Mat unit, a Tactical Rescue unit, a Dive Rescue unit, two Marine units, and numerous special, support, and reserve units. John J. Gelinas, Chief of Operations, is in charge
  of day to day operation of the department.<ref>{{cite web|url=http://www2.cambridgema.gov/cfd/ |title=City of Cambridge Fire Department |publisher=.cambridgema.gov |date=2005-03-13 |accessdate=2012-06-26}}</ref> The CFD is rated as a Class 1 fire department by the [[Insurance Services Office]] (ISO), and is one of only 32 fire departments so rated, out of 37,000 departments in the United States. The other class 1 departments in New England are in [[Hartford, Connecticut]] and [[Milford, Connecticut]]. Class 1 signifies the highest level of fire protection according to various criteria.<ref>{{cite web|url=http://www2.cambridgema.gov/CFD/Class1FD.cfm |title=Class 1 Fire Department |publisher=.cambridgema.gov |date=1999-07-01 |accessdate=2012-06-26}}</ref> The CFD responds to approximately 15,000 emergency calls annually. {| class=wikitable |- valign=bottom ! Engine Company ! Ladder Company ! Special Unit ! Division ! Address ! Neighborhood |- | Engine 1 || Ladder 1 || || || 491 Broad
 way || Harvard Square |- | Engine 2 || Ladder 3 || Squad 2 || || 378 Massachusetts Ave. || Lafayette Square |- | Engine 3 || Ladder 2 || || || 175 Cambridge St. || East Cambridge |- | Engine 4 || || Squad 4 || || 2029 Massachusetts Ave. || Porter Square |- | Engine 5 || || || Division 1 || 1384 Cambridge St. || Inman Square |- | Engine 6 || || || || 176 River St. || Cambridgeport |- | Engine 8 || Ladder 4 || || Division 2 || 113 Garden St. || Taylor Square |- | Engine 9 || || || || 167 Lexington Ave || West Cambridge |- | Maintenance Facility || || || || 100 Smith Pl. || |} ===Water Department=== Cambridge is unusual among cities inside Route 128 in having a non-[[MWRA]] water supply. City water is obtained from [[Hobbs Brook]] (in [[Lincoln, Massachusetts|Lincoln]] and [[Waltham, Massachusetts|Waltham]]), [[Stony Brook (Boston)|Stony Brook]] (Waltham and [[Weston, Massachusetts|Weston]]), and [[Fresh Pond (Cambridge, Massachusetts)|Fresh Pond]] (Cambridge). The city owns over 1200 
 acres of land in other towns that includes these reservoirs and portions of their watershed.<ref>{{cite web|url=http://www2.cambridgema.gov/CWD/wat_lands.cfm |title=Cambridge Watershed Lands & Facilities |publisher=.cambridgema.gov |date= |accessdate=2012-04-28}}</ref> Water is treated at Fresh Pond, then pumped uphill to an elevation of {{convert|176|ft|m}} [[above sea level]] at the Payson Park Reservoir ([[Belmont, Massachusetts|Belmont]]); From there, the water is redistributed downhill via gravity to individual users in the city.<ref>{{cite web|url=http://www.cambridgema.gov/CityOfCambridge_Content/documents/CWD_March_2010.pdf |title=Water supply system |format=PDF |date= |accessdate=2012-04-28}}</ref><ref>[http://www.cambridgema.gov/CWD/fpfaqs.cfm Is Fresh Pond really used for drinking water?], Cambridge Water Department</ref> ===County government=== Cambridge is a [[county seat]] of [[Middlesex County, Massachusetts]], along with [[Lowell, Massachusetts|Lowell]]. Though the c
 ounty government was abolished in 1997, the county still exists as a geographical and political region. The employees of Middlesex County courts, jails, registries, and other county agencies now work directly for the state. At present, the county's registrars of [[Deed]]s and Probate remain in Cambridge; however, the Superior Court and District Attorney have had their base of operations transferred to [[Woburn, Massachusetts|Woburn]]. Third District court has shifted operations to [[Medford, Massachusetts|Medford]], and the Sheriff's office for the county is still awaiting a near-term relocation.<ref>{{cite news | url=http://www.boston.com/news/local/massachusetts/articles/2008/02/14/court_move_a_hassle_for_commuters/ |title=Court move a hassle for commuters |accessdate=July 25, 2009 |first=Eric |last=Moskowitz |authorlink= |coauthors= |date=February 14, 2008 |work=[[Boston Globe|The Boston Globe]] |pages= |archiveurl= |archivedate= |quote=In a little more than a month, Middlesex Su
 perior Court will open in Woburn after nearly four decades at the Edward J. Sullivan Courthouse in Cambridge. With it, the court will bring the roughly 500 people who pass through its doors each day \u2013 the clerical staff, lawyers, judges, jurors, plaintiffs, defendants, and others who use or work in the system.}}</ref><ref>{{cite news | url=http://www.wickedlocal.com/cambridge/homepage/x135741754/Cambridges-Middlesex-Jail-courts-may-be-shuttered-for-good |title=Cambridge's Middlesex Jail, courts may be shuttered for good |accessdate=July 25, 2009 |first=Charlie |last=Breitrose |authorlink= |coauthors= |date=July 7, 2009 |work=Wicked Local News: Cambridge |pages= |archiveurl= |archivedate= |quote=The courts moved out of the building to allow workers to remove asbestos. Superior Court moved to Woburn in March 2008, and in February, the Third District Court moved to Medford.}}</ref> ==Education== [[File:MIT Main Campus Aerial.jpg|thumb|Aerial view of part of [[MIT]]'s main campus]] [[
 File:Dunster House.jpg|thumb|[[Dunster House]], Harvard]] ===Higher education=== Cambridge is perhaps best known as an academic and intellectual center, owing to its colleges and universities, which include: *[[Cambridge College]] *[[Cambridge School of Culinary Arts]] *[[Episcopal Divinity School]] *[[Harvard University]] *[[Hult International Business School]] *[[Lesley University]] *[[Longy School of Music]] *[[Massachusetts Institute of Technology]] *[[Le Cordon Bleu College of Culinary Arts in Boston]] [[Nobel laureates by university affiliation|At least 129]] of the world's total 780 [[Nobel Prize]] winners have been, at some point in their careers, affiliated with universities in Cambridge. The [[American Academy of Arts and Sciences]] is also based in Cambridge. ===Primary and secondary public education=== The Cambridge Public School District encompasses 12 elementary schools that follow a variety of different educational systems and philosophies. All but one of the elementa
 ry schools extend up to the [[middle school]] grades as well. The 12 elementary schools are: *[[Amigos School]] *Baldwin School *Cambridgeport School *Fletcher-Maynard Academy *Graham and Parks Alternative School *Haggerty School *Kennedy-Longfellow School *King Open School *Martin Luther King, Jr. School *Morse School (a [[Core Knowledge Foundation|Core Knowledge]] school) *Peabody School *Tobin School (a [[Montessori school]]) There are three public high schools serving Cambridge students, including the [[Cambridge Rindge and Latin School]].<ref>{{cite web|url=http://www.cpsd.us/Web/PubInfo/SchoolsAtAGlance06-07.pdf|title=Cambridge Public Schools at a Glance|format=PDF}}{{dead link|date=June 2012}}</ref> and Community Charter School of Cambridge (www.ccscambridge.org) In 2003, the CRLS, also known as Rindge, came close to losing its educational accreditation when it was placed on probation by the [[New England Association of Schools and Colleges]].<ref name="Crimson MCAS">{{cite w
 eb|url=http://www.thecrimson.com/article.aspx?ref=512061|title=School Fights Achievement Gap|publisher=The Harvard Crimson|accessdate=May 14, 2009}}</ref> The school has improved under Principal Chris Saheed, graduation rates hover around 98%, and 70% of students gain college admission. Community Charter School of Cambridge serves 350 students, primarily from Boston and Cambridge, and is a tuition free public charter school with a college preparatory curriculum. All students from the class of 2009 and 2010 gained admission to college. Outside of the main public schools are public charter schools including: [[Benjamin Banneker Charter School]], which serves students in grades K-6,<ref>{{cite web|url=http://www.banneker.org/ |title=The Benjamin Banneker Charter Public School |publisher=Banneker.org |date=2012-03-01 |accessdate=2012-04-28}}</ref> [[Community Charter School of Cambridge]],<ref>{{cite web|url=http://www.ccscambridge.org/ |title=Community Charter School of Cambridge |publ
 isher=Ccscambridge.org |date= |accessdate=2012-04-28}}</ref> which is located in Kendall Square and serves students in grades 7\u201312, and [[Prospect Hill Academy]], a [[charter school]] whose upper school is in [[Central Square (Cambridge)|Central Square]], though it is not a part of the Cambridge Public School District. ===Primary and secondary private education=== [[File:Cambridge Public Library, Cambridge, Massachusetts.JPG|thumb|right|[[Cambridge Public Library]] original building, part of an expanded facility]] There are also many private schools in the city including: <!-- please keep alphabetical --> *[[Boston Archdiocesan Choir School]] (BACS) *[[Buckingham Browne & Nichols]] (BB&N) *[[Cambridge montessori school|Cambridge Montessori School]] (CMS) *Cambridge [[Religious Society of Friends|Friends]] School. Thomas Waring served as founding headmaster of the school. *Fayerweather Street School (FSS)[http://www.fayerweather.org/ ] *[[International School of Boston]] (ISB, form
 erly �cole Bilingue) *[[Matignon High School]] *[[North Cambridge Catholic High School]] (re-branded as Cristo Rey Boston and relocated to Dorchester, MA in 2010) *[[Shady Hill School]] *St. Peter School ==Economy== [[File:Cambridge Skyline.jpg|thumb|Buildings of [[Kendall Square]], center of Cambridge's [[biotech]] economy, seen from the [[Charles River]]]] Manufacturing was an important part of the economy in the late 19th and early 20th century, but educational institutions are the city's biggest employers today. Harvard and [[Massachusetts Institute of Technology|MIT]] together employ about 20,000.<ref name="2008top25">[http://www2.cambridgema.gov/cdd/data/labor/top25/top25_2008.html Top 25 Cambridge Employers: 2008], City of Cambridge</ref> As a cradle of technological innovation, Cambridge was home to technology firms [[Analog Devices]], [[Akamai Technologies|Akamai]], [[BBN Technologies|Bolt, Beranek, and Newman (BBN Technologies)]] (now part of Raytheon), [[General Radio|Ge
 neral Radio (later GenRad)]], [[Lotus Development Corporation]] (now part of [[IBM]]), [[Polaroid Corporation|Polaroid]], [[Symbolics]], and [[Thinking Machines]]. In 1996, [[Polaroid Corporation|Polaroid]], [[Arthur D. Little]], and [[Lotus Development Corporation|Lotus]] were top employers with over 1,000 employees in Cambridge, but faded out a few years later. Health care and biotechnology firms such as [[Genzyme]], [[Biogen Idec]], [[Millennium Pharmaceuticals]], [[Sanofi]], [[Pfizer]] and [[Novartis]]<ref>{{cite news |title=Novartis doubles plan for Cambridge |author=Casey Ross and Robert Weisman |first= |last= |authorlink= |authorlink2= |url=http://articles.boston.com/2010-10-27/business/29323650_1_french-drug-maker-astrazeneca-plc-research-operations |agency= |newspaper=[[The Boston Globe]] |publisher= |isbn= |issn= |pmid= |pmd= |bibcode= |doi= |date=October 27, 2010 |page= |pages= |accessdate=April 12, 2011|quote=Already Cambridge\u2019s largest corporate employer, the Swiss fi
 rm expects to hire an additional 200 to 300 employees over the next five years, bringing its total workforce in the city to around 2,300. Novartis\u2019s global research operations are headquartered in Cambridge, across Massachusetts Avenue from the site of the new four-acre campus. |archiveurl= |archivedate= |ref=}}</ref> have significant presences in the city. Though headquartered in Switzerland, Novartis continues to expand its operations in Cambridge. Other major biotech and pharmaceutical firms expanding their presence in Cambridge include [[GlaxoSmithKline]], [[AstraZeneca]], [[Shire plc|Shire]], and [[Pfizer]].<ref>{{cite news|title=Novartis Doubles Plan for Cambridge|url=http://www.boston.com/business/healthcare/articles/2010/10/27/novartis_doubles_plan_for_cambridge/|accessdate=23 February 2012 | work=The Boston Globe|first1=Casey|last1=Ross|first2=Robert|last2=Weisman|date=October 27, 2010}}</ref> Most Biotech firms in Cambridge are located around [[Kendall Square]] and [[Eas
 t Cambridge, Massachusetts|East Cambridge]], which decades ago were the city's center of manufacturing. A number of biotechnology companies are also located in [[University Park at MIT]], a new development in another former manufacturing area. None of the high technology firms that once dominated the economy was among the 25 largest employers in 2005, but by 2008 high tech companies [[Akamai Technologies|Akamai]] and [[ITA Software]] had grown to be among the largest 25 employers.<ref name="2008top25" /> [[Google]],<ref>{{cite web|url=http://www.google.com/corporate/address.html |title=Google Offices |publisher=Google.com |date= |accessdate=2012-07-18}}</ref> [[IBM Research]], and [[Microsoft Research]] maintain offices in Cambridge. In late January 2012\u2014less than a year after acquiring [[Billerica, Massachusetts|Billerica]]-based analytic database management company, [[Vertica]]\u2014[[Hewlett-Packard]] announced it would also be opening its first offices in Cambridge.<ref>{{cite we
 b|last=Huang|first=Gregory|title=Hewlett-Packard Expands to Cambridge via Vertica\u2019s "Big Data" Center|url=http://www.xconomy.com/boston/2012/01/23/hewlett-packard-expands-to-cambridge-via-verticas-big-data-center/?single_page=true}}</ref> Around this same time, e-commerce giants [[Staples Inc.|Staples]]<ref>{{cite web|title=Staples to bring e-commerce office to Cambridge's Kendall Square Read more: Staples to bring e-commerce office to Cambridge's Kendall Square - Cambridge, Massachusetts - Cambridge Chronicle http://www.wickedlocal.com/cambridge/news/x690035936/Staples-to-bring-E-commerce-office-to-Cambridges-Kendall-Square#ixzz1nDY39Who|url=http://www.wickedlocal.com/cambridge/news/x690035936/Staples-to-bring-E-commerce-office-to-Cambridges-Kendall-Square#axzz1kg3no7Zg}}</ref> and [[Amazon.com]]<ref>{{cite web|title=Amazon Seeks Brick-And-Mortar Presence In Boston Area|url=http://www.wbur.org/2011/12/22/amazon-boston}}</ref> said they would be opening research and innovation cen
 ters in Kendall Square. Video game developer [[Harmonix Music Systems]] is based in [[Central Square (Cambridge)|Central Square]]. The proximity of Cambridge's universities has also made the city a center for nonprofit groups and think tanks, including the [[National Bureau of Economic Research]], the [[Smithsonian Astrophysical Observatory]], the [[Lincoln Institute of Land Policy]], [[Cultural Survival]], and [[One Laptop per Child]]. In September 2011, an initiative by the City of Cambridge called the "[[Entrepreneur Walk of Fame]]" was launched. It seeks to highlight individuals who have made contributions to innovation in the global business community.<ref>{{cite news |title=Stars of invention |author= |first=Kathleen |last=Pierce |url=http://articles.boston.com/2011-09-16/business/30165912_1_gates-and-jobs-microsoft-granite-stars |agency= |newspaper=The Boston Globe|date=September 16, 2011 |page= |pages= |at= |accessdate=October 1, 2011}}</ref> ===Top employers=== The top ten 
 employers in the city are:<ref>{{cite web|url=http://cambridgema.gov/citynewsandpublications/news/2012/01/fy11comprehensiveannualfinancialreportnowavailable.aspx |title=City of Cambridge, Massachusetts Comprehensive Annual Financial Report July 1, 2010\u2014June 30, 2011 |publisher=Cambridgema.gov |date=2011-06-30 |accessdate=2012-04-28}}</ref> {| class="wikitable" |- ! # ! Employer ! # of employees |- | 1 |[[Harvard University]] |10,718 |- |2 |[[Massachusetts Institute of Technology]] |7,604 |- |3 |City of Cambridge |2,922 |- |4 |[[Novartis]] Institutes for BioMedical Research |2,095 |- |5 |[[Mount Auburn Hospital]] |1,665 |- |6 |[[Vertex Pharmaceuticals]] |1,600 |- |7 |[[Genzyme]] |1,504 |- |8 |[[Biogen Idec]] |1,350 |- |9 |[[Federal government of the United States|Federal Government]] |1,316 |- |10 |[[Pfizer]] |1,300 |} ==Transportation== {{See also|Boston transportation}} ===Road=== [[File:Harvard Square at Peabody Street and Mass Avenue.jpg|thumb|[[Massachusetts Avenue (Boston)|Ma
 ssachusetts Avenue]] in [[Harvard Square]]]] Several major roads lead to Cambridge, including [[Massachusetts State Highway 2|Route 2]], [[Massachusetts State Highway 16|Route 16]] and the [[Massachusetts State Highway 28|McGrath Highway (Route 28)]]. The [[Massachusetts Turnpike]] does not pass through Cambridge, but provides access by an exit in nearby [[Allston, Massachusetts|Allston]]. Both [[U.S. Route 1]] and [[I-93 (MA)]] also provide additional access on the eastern end of Cambridge at Leverett Circle in [[Boston]]. [[Massachusetts State Highway 2A|Route 2A]] runs the length of the city, chiefly along Massachusetts Avenue. The Charles River forms the southern border of Cambridge and is crossed by 11 bridges connecting Cambridge to Boston, including the [[Longfellow Bridge]] and the [[Harvard Bridge]], eight of which are open to motorized road traffic. Cambridge has an irregular street network because many of the roads date from the colonial era. Contrary to popular belief, t
 he road system did not evolve from longstanding cow-paths. Roads connected various village settlements with each other and nearby towns, and were shaped by geographic features, most notably streams, hills, and swampy areas. Today, the major "squares" are typically connected by long, mostly straight roads, such as Massachusetts Avenue between [[Harvard Square]] and [[Central Square (Cambridge)|Central Square]], or Hampshire Street between [[Kendall Square]] and [[Inman Square]]. ===Mass transit=== [[File:Central MBTA station.jpg|thumb|[[Central (MBTA)|Central station on the MBTA Red Line]]]] Cambridge is well served by the [[MBTA]], including the [[Porter (MBTA station)|Porter Square stop]] on the regional [[MBTA Commuter Rail|Commuter Rail]], the [[Lechmere (MBTA station)|Lechmere stop]] on the [[Green Line (MBTA)|Green Line]], and five stops on the [[Red Line (MBTA)|Red Line]] ([[Alewife Station (MBTA)|Alewife]], [[Porter (MBTA)|Porter Square]], [[Harvard (MBTA station)|Harvard Squ
 are]], [[Central (MBTA station)|Central Square]], and [[Kendall/MIT (MBTA station)|Kendall Square/MIT]]). Alewife Station, the current terminus of the Red Line, has a large multi-story parking garage (at a rate of $7 per day {{as of|lc=y|2009}}).<ref>{{cite web|url=http://www.mbta.com/schedules_and_maps/subway/lines/stations/?stopId=10029 |title=> Schedules & Maps > Subway > Alewife Station |publisher=MBTA |date= |accessdate=2012-04-28}}</ref> The [[Harvard Bus Tunnel]], under Harvard Square, reduces traffic congestion on the surface, and connects to the Red Line underground. This tunnel was originally opened for streetcars in 1912, and served trackless trolleys and buses as the routes were converted. The tunnel was partially reconfigured when the Red Line was extended to Alewife in the early 1980s. Outside of the state-owned transit agency, the city is also served by the Charles River Transportation Management Agency (CRTMA) shuttles which are supported by some of the largest compa
 nies operating in city, in addition to the municipal government itself.<ref>{{cite web |url=http://www.charlesrivertma.org/members.htm |title=Charles River TMA Members |author=Staff writer |date=(As of) January 1, 2013 |work=CRTMA |publisher= |language= |trans_title= |type= |archiveurl= |archivedate= |deadurl= |accessdate=January 1, 2013 |quote= |ref= |separator= |postscript=}} </ref> ===Cycling=== Cambridge has several [[bike path]]s, including one along the Charles River,<ref>{{cite web|url=http://www.mass.gov/dcr/parks/metroboston/maps/bikepaths_dudley.gif |title=Dr. Paul Dudley White Bikepath |date= |accessdate=2012-04-28}}</ref> and the [[Cambridge Linear Park|Linear Park]] connecting the [[Minuteman Bikeway]] at Alewife with the [[Somerville Community Path]]. Bike parking is common and there are bike lanes on many streets, although concerns have been expressed regarding the suitability of many of the lanes. On several central MIT streets, bike lanes transfer onto the sidewalk.
  Cambridge bans cycling on certain sections of sidewalk where pedestrian traffic is heavy.<ref>{{cite web|url=http://www.cambridgema.gov/cdd/et/bike/bike_ban.html |title=Sidewalk Bicycling Banned Areas \u2013 Cambridge Massachusetts |publisher=Cambridgema.gov |date= |accessdate=2012-04-28}}</ref><ref>{{cite web|url=http://www.cambridgema.gov/cdd/et/bike/bike_reg.html |title=Traffic Regulations for Cyclists \u2013 Cambridge Massachusetts |publisher=Cambridgema.gov |date=1997-05-01 |accessdate=2012-04-28}}</ref> While ''[[Bicycling Magazine]]'' has rated Boston as one of the worst cities in the nation for bicycling (In their words, for "lousy roads, scarce and unconnected bike lanes and bike-friendly gestures from City Hall that go nowhere\u2014such as hiring a bike coordinator in 2001, only to cut the position two years later"),<ref>[http://www.bicycling.com/article/1,6610,s1-2-16-14593-11,00.html Urban Treasures \u2013 bicycling.com]{{dead link|date=April 2012}}</ref> it has listed Cambridge as
  an honorable mention as one of the best<ref>[http://www.bicycling.com/article/1,6610,s1-2-16-14593-9,00.html Urban Treasures \u2013 bicycling.com]{{dead link|date=April 2012}}</ref> and was called by the magazine "Boston's Great Hope." Cambridge has an active, official bicycle committee. ===Walking=== [[File:Weeks Footbridge Cambridge, MA.jpg|thumb|The [[John W. Weeks Bridge|Weeks Bridge]] provides a pedestrian-only connection between Boston's Allston-Brighton neighborhood and Cambridge over the Charles River]] Walking is a popular activity in Cambridge. Per year 2000 data, of the communities in the U.S. with more than 100,000 residents, Cambridge has the highest percentage of commuters who walk to work.<ref>{{cite web|url=http://www.bikesatwork.com/carfree/census-lookup.php?state_select=ALL_STATES&lower_pop=100000&upper_pop=99999999&sort_num=2&show_rows=25&first_row=0 |title=The Carfree Census Database: Result of search for communities in any state with population over 100,000, sorte
 d in descending order by % Pedestrian Commuters |publisher=Bikesatwork.com |date= |accessdate=2012-04-28}}</ref> Cambridge receives a "Walk Score" of 100 out of 100 possible points.<ref>[http://www.walkscore.com/get-score.php?street=cambridge%2C+ma&go=Go Walk Score site] Accessed July 28, 2009</ref> Cambridge's major historic squares have been recently changed into a modern walking landscape, which has sparked a traffic calming program based on the needs of pedestrians rather than of motorists. ===Intercity=== The Boston intercity bus and train stations at [[South Station]], Boston, and [[Logan International Airport]] in [[East Boston]], are accessible by [[Red Line (MBTA)|subway]]. The [[Fitchburg Line]] rail service from [[Porter (MBTA station)|Porter Square]] connects to some western suburbs. Since October 2010, there has also been intercity bus service between [[Alewife (MBTA station)|Alewife Station]] (Cambridge) and [[New York City]].<ref>{{cite web|last=Thomas |first=Sarah |u
 rl=http://www.boston.com/yourtown/news/cambridge/2010/10/warren_mbta_welcome_world_wide.html |title=NYC-bound buses will roll from Newton, Cambridge |publisher=Boston.com |date=2010-10-19 |accessdate=2012-04-28}}</ref> ==Media== ===Newspapers=== Cambridge is served by several weekly newspapers. The most prominent is the ''[[Cambridge Chronicle]]'', which is also the oldest surviving weekly paper in the United States. ===Radio=== Cambridge is home to the following commercially licensed and student-run radio stations: {| class=wikitable |- ! [[Callsign]] !! Frequency !! City/town !! Licensee !! Format |- | [[WHRB]] || align=right | 95.3 FM || Cambridge (Harvard) || Harvard Radio Broadcasting Co., Inc. || [[Variety (US radio)|Musical variety]] |- | [[WJIB]] || align=right | 740&nbsp;AM || Cambridge || Bob Bittner Broadcasting || [[Adult Standards]]/Pop |- | [[WMBR]] || align=right | 88.1 FM || Cambridge (MIT) || Technology Broadcasting Corporation || [[College radio]] |} ===Television=
 == Cambridge Community Television (CCTV) has served the Cambridge community since its inception in 1988. CCTV operates Cambridge's public access television facility and programs three television channels, 8, 9, and 96 on the Cambridge cable system (Comcast). ===Social media=== As of 2011, a growing number of social media efforts provide means for participatory engagement with the locality of Cambridge, such as Localocracy<ref>"Localocracy is an online town common where registered voters using real names can weigh in on local issues." [http://cambridge.localocracy.com/ Localocracy Cambridge, Massachusetts]. Accessed 2011-10-01</ref> and [[foursquare (website)|Foursquare]]. ==Culture, art and architecture== [[File:Fogg.jpg|thumb|[[Fogg Museum]], Harvard]] ===Museums=== * [[Harvard Art Museum]], including the [[Busch-Reisinger Museum]], a collection of Germanic art the [[Fogg Art Museum]], a comprehensive collection of Western art, and the [[Arthur M. Sackler Museum]], a collection of 
 Middle East and Asian art * [[Harvard Museum of Natural History]], including the [[Glass Flowers]] collection * [[Peabody Museum of Archaeology and Ethnology]], Harvard *[[Semitic Museum]], Harvard * [[MIT Museum]] * [[List Visual Arts Center]], MIT ===Public art=== Cambridge has a large and varied collection of permanent public art, both on city property (managed by the Cambridge Arts Council),<ref>{{cite web|url=http://www.cambridgema.gov/CAC/Public/overview.cfm |title=CAC Public Art Program |publisher=Cambridgema.gov |date=2007-03-13 |accessdate=2012-04-28}}</ref> and on the campuses of Harvard<ref>{{cite web|url=http://ofa.fas.harvard.edu/visualarts/pubart.php |title=Office for the Arts at Harvard: Public Art |publisher=Ofa.fas.harvard.edu |date= |accessdate=2012-04-28}}</ref> and MIT.<ref>{{cite web|url=http://listart.mit.edu/map |title=MIT Public Art Collection Map |publisher=Listart.mit.edu |date= |accessdate=2012-04-28}}</ref> Temporary public artworks are displayed as part 
 of the annual Cambridge River Festival on the banks of the Charles River, during winter celebrations in Harvard and Central Squares, and at university campus sites. Experimental forms of public artistic and cultural expression include the Central Square World's Fair, the Somerville-based annual Honk! Festival,<ref>{{cite web|url=http://honkfest.org/ |title= Honk Fest}}</ref> and [[If This House Could Talk]],<ref>{{cite web|url=http://cambridgehistory.org/discover/ifthishousecouldtalk/index.html |title=The Cambridge Historical Society}}</ref> a neighborhood art and history event. {{or|date=April 2012}} {{Citation needed|date=April 2012}} An active tradition of street musicians and other performers in Harvard Square entertains an audience of tourists and local residents during the warmer months of the year. The performances are coordinated through a public process that has been developed collaboratively by the performers,<ref>{{cite web|url=http://www.buskersadvocates.org/ | title= St
 reet Arts & Buskers Advocates}}</ref> city administrators, private organizations and business groups.<ref>{{cite web|url=http://harvardsquare.com/Home/Arts-and-Entertainment/Street-Arts-and-Buskers-Advocates.aspx |title=Street Arts and Buskers Advocates |publisher=Harvardsquare.com |date= |accessdate=2012-04-28}}</ref> [[File:Longfellow National Historic Site, Cambridge, Massachusetts.JPG|thumb|right|The [[Longfellow National Historic Site]]]] [[File:Wfm stata center.jpg|thumb|[[Stata Center]], MIT]] [[File:Simmons Hall, MIT, Cambridge, Massachusetts.JPG|thumb|[[List of MIT undergraduate dormitories|Simmons Hall]], MIT]] ===Architecture=== Despite intensive urbanization during the late 19th century and 20th century, Cambridge has preserved an unusual number of historic buildings, including some dating to the 17th century. The city also contains an abundance of innovative contemporary architecture, largely built by Harvard and MIT. ;Notable historic buildings in the city include: * T
 he [[Asa Gray House]] (1810) * [[Austin Hall, Harvard University]] (1882\u201384) * [[Cambridge, Massachusetts City Hall|Cambridge City Hall]] (1888\u201389) * [[Cambridge Public Library]] (1888) * [[Christ Church, Cambridge]] (1761) * [[Cooper-Frost-Austin House]] (1689\u20131817) * [[Elmwood (Cambridge, Massachusetts)|Elmwood House]] (1767), residence of the [[President of Harvard University]] * [[First Church of Christ, Scientist (Cambridge, Massachusetts)|First Church of Christ, Scientist]] (1924\u201330) * [[The First Parish in Cambridge]] (1833) * [[Harvard-Epworth United Methodist Church]] (1891\u201393) * [[Harvard Lampoon Building]] (1909) * The [[Hooper-Lee-Nichols House]] (1685\u20131850) * [[Longfellow National Historic Site]] (1759), former home of poet [[Henry Wadsworth Longfellow]] * [[The Memorial Church of Harvard University]] (1932) * [[Memorial Hall, Harvard University]] (1870\u201377) * [[Middlesex County Courthouse (Massachusetts)|Middlesex County Courthouse]] (1814\u201348) * [[Urban 
 Rowhouse (40-48 Pearl Street, Cambridge, Massachusetts)|Urban Rowhouse]] (1875) * [[spite house|O'Reilly Spite House]] (1908), built to spite a neighbor who would not sell his adjacent land<ref name="existing">Bloom, Jonathan. (February 2, 2003) [[Boston Globe]] ''[http://nl.newsbank.com/nl-search/we/Archives?p_product=BG&p_theme=bg&p_action=search&p_maxdocs=200&p_topdoc=1&p_text_direct-0=0F907F2342522B5D&p_field_direct-0=document_id&p_perpage=10&p_sort=YMD_date:D Existing by the Thinnest of Margins. A Concord Avenue Landmark Gives New Meaning to Cozy.]'' Section: City Weekly; Page 11. Location: 260 Concord Ave, Cambridge, MA 02138.</ref> {{See also|List of Registered Historic Places in Cambridge, Massachusetts}} ;Contemporary architecture: * [[List of MIT undergraduate dormitories#Baker House|Baker House]] dormitory, MIT, by Finnish architect [[Alvar Aalto]], one of only two buildings by Aalto in the US * Harvard Graduate Center/Harkness Commons, by [[The Architects Collaborative]]
  (TAC, with [[Walter Gropius]]) * [[Carpenter Center for the Visual Arts]], Harvard, the only building in North America by [[Le Corbusier]] * [[Kresge Auditorium]], MIT, by [[Eero Saarinen]] * [[MIT Chapel]], by [[Eero Saarinen]] * [[Design Research Building]], by [[Benjamin Thompson and Associates]] * [[American Academy of Arts and Sciences]], by [[Kallmann McKinnell and Wood]], also architects of Boston City Hall * [[Arthur M. Sackler Museum]], Harvard, one of the few buildings in the U.S. by [[James Stirling (architect)|James Stirling]], winner of the [[Pritzker Prize]] * [[Stata Center]], MIT, by [[Frank Gehry]] * [[List of MIT undergraduate dormitories#Simmons Hall|Simmons Hall]], MIT, by [[Steven Holl]] ===Music=== <!-- make section generic. NEEDS MORE WORK. remove marketing fluff for Ryles. --> The city has an active music scene from classical performances to the latest popular bands. ==Sister cities== Cambridge has 8 active, official [[Twin towns and sister cities|sister cit
 ies]], and an unofficial relationship with [[Cambridge]], England:<ref name="peacom">"A message from the Peace Commission" [http://www.cambridgema.gov/peace/newsandpublications/news/detail.aspx?path=%2fsitecore%2fcontent%2fhome%2fpeace%2fnewsandpublications%2fnews%2f2008%2f02%2finformationoncambridgessistercities].</ref> *{{Flagicon|PRT}} [[Coimbra]], [[Portugal]] *{{Flagicon|CUB}} [[Cienfuegos]], [[Cuba]] *{{Flagicon|ITA}} [[Gaeta]], [[Italy]] *{{Flagicon|IRL}} [[Galway]], [[Republic of Ireland|Ireland]] *{{Flagicon|ARM}} [[Yerevan]], [[Armenia]]<ref>{{cite web|url=http://www.cysca.org/ |title=Cambridge-Yerevan Sister City Association |publisher=Cysca.org |date= |accessdate=2012-04-28}}</ref> *{{Flagicon|SLV}} [[San Jos� Las Flores, Chalatenango|San Jos� Las Flores]], [[El Salvador]] *{{Flagicon|JPN}} [[Tsukuba, Ibaraki|Tsukuba Science City]], Japan *{{Flagicon|POL}} [[Krak�w]], [[Poland]] *{{Flagicon|CHN}} [[Haidian District]], [[China]] Ten other official sister city relations
 hips are inactive: [[Dublin]], Ireland; [[Ischia]], [[Catania]], and [[Florence]], Italy; [[Krak�w]], Poland; [[Santo Domingo Oeste]], Dominican Republic; [[Southwark]], London, England; [[Yuseong]], Daejeon, Korea; and [[Haidian District|Haidian]], Beijing, China.<ref name="peacom"/> There has also been an unofficial relationship with: *{{Flagicon|GBR}} [[Cambridge]], England, UK<ref>{{cite web|url=http://www.cambridgema.gov/peace/newsandpublications/news/detail.aspx?path=%2fsitecore%2fcontent%2fhome%2fpeace%2fnewsandpublications%2fnews%2f2008%2f02%2finformationoncambridgessistercities |title="Sister Cities", Cambridge Peace Commission |publisher=Cambridgema.gov |date=2008-02-15 |accessdate=2012-07-18}}</ref> ==Zip codes== *02138\u2014Harvard Square/West Cambridge *02139\u2014Central Square/Inman Square/MIT *02140\u2014Porter Square/North Cambridge *02141\u2014East Cambridge *02142\u2014Kendall Square ==References== {{reflist|30em}} ==General references== * ''History of Middlesex County, Massach
 usetts'', [http://books.google.com/books?id=QGolOAyd9RMC&dq=intitle:History+intitle:of+intitle:Middlesex+intitle:County+intitle:Massachusetts&lr=&num=50&as_brr=0&source=gbs_other_versions_sidebar_s&cad=5 Volume 1 (A-H)], [http://books.google.com/books?id=hNaAnwRMedUC&pg=PA506&dq=intitle:History+intitle:of+intitle:Middlesex+intitle:County+intitle:Massachusetts&lr=&num=50&as_brr=0#PPA3,M1 Volume 2 (L-W)] compiled by Samuel Adams Drake, published 1879\u20131880. ** [http://books.google.com/books?id=QGolOAyd9RMC&printsec=titlepage#PPA305,M1 Cambridge article] by Rev. Edward Abbott in volume 1, pages 305\u2013358. *Eliot, Samuel Atkins. ''A History of Cambridge, Massachusetts: 1630\u20131913''. Cambridge: The Cambridge Tribune, 1913. *Hiestand, Emily. "Watershed: An Excursion in Four Parts" The Georgia Review Spring 1998 pages 7\u201328 *[[Lucius Robinson Paige|Paige, Lucius]]. ''History of Cambridge, Massachusetts: 1630\u20131877''. Cambridge: The Riverside Press, 1877. *Survey of Architectural Histor
 y in Cambridge: Mid Cambridge, 1967, Cambridge Historical Commission, Cambridge, Mass.{{ISBN missing}} *Survey of Architectural History in Cambridge: Cambridgeport, 1971 ISBN 0-262-53013-9, Cambridge Historical Commission, Cambridge, Mass. *Survey of Architectural History in Cambridge: Old Cambridge, 1973 ISBN 0-262-53014-7, Cambridge Historical Commission, Cambridge, Mass. *Survey of Architectural History in Cambridge: Northwest Cambridge, 1977 ISBN 0-262-53032-5, Cambridge Historical Commission, Cambridge, Mass. *Survey of Architectural History in Cambridge: East Cambridge, 1988 (revised) ISBN 0-262-53078-3, Cambridge Historical Commission, Cambridge, Mass. *{{cite book|last=Sinclair|first=Jill|title=Fresh Pond: The History of a Cambridge Landscape|publisher=MIT Press|location=Cambridge, Mass.|date=April 2009|isbn=978-0-262-19591-1 }} *{{cite book|last=Seaburg|first=Alan|title=Cambridge on the Charles|url=http://books.google.com/books?id=c7_oCS782-8C|publisher=Anne Miniver Press|l
 ocation=Billerica, Mass.|year=2001|author=Seaburg, A. and Dahill, T. and Rose, C.H.|isbn=978-0-9625794-9-3}} ==External links== {{Commons category}} <!-- for current and future use if material is uploaded --> {{Wikivoyage|Cambridge (Massachusetts)}} {{Portal|Boston}} {{Commons category|Cambridge, Massachusetts}} *{{Official website|http://www.cambridgema.gov/}} *[http://www.cambridge-usa.org/ Cambridge Office for Tourism] *[http://www.city-data.com/city/Cambridge-Massachusetts.html City-Data.com] *[http://www.epodunk.com/cgi-bin/genInfo.php?locIndex=2894 ePodunk: Profile for Cambridge, Massachusetts] *{{dmoz|Regional/North_America/United_States/Massachusetts/Localities/C/Cambridge}} <br/><!--this break is to put visual space between the last information and the following template if needed--> ===Maps=== *[http://www.cambridgema.gov/GIS/FindMapAtlas.cfm Cambridge Maps] *[http://www.cambridgema.gov/GIS City of Cambridge Geographic Information System (GIS)] *[http://www.salemdeeds.com/
 atlases_results.asp?ImageType=index&atlastype=MassWorld&atlastown=&atlas=MASSACHUSETTS+1871&atlas_desc=MASSACHUSETTS+1871 ''1871 Atlas of Massachusetts''.] by Wall & Gray. [http://www.salemdeeds.com/atlases_pages.asp?ImageName=PAGE_0010_0011.jpg&atlastype=MassWorld&atlastown=&atlas=MASSACHUSETTS+1871&atlas_desc=MASSACHUSETTS+1871&pageprefix= Map of Massachusetts.] [http://www.salemdeeds.com/atlases_pages.asp?ImageName=PAGE_0044_0045.jpg&atlastype=MassWorld&atlastown=&atlas=MASSACHUSETTS+1871&atlas_desc=MASSACHUSETTS+1871&pageprefix= Map of Middlesex County.] *Dutton, E.P. [http://maps.bpl.org/details_10717/?srch_query=Dutton%2C+E.P.&srch_fields=all&srch_author=on&srch_style=exact&srch_fa=save&srch_ok=Go+Search Chart of Boston Harbor and Massachusetts Bay with Map of Adjacent Country.] Published 1867. A good map of roads and rail lines around Cambridge. *[http://www.citymap.com/cambridge/index.htm Cambridge Citymap \u2013 Community, Business, and Visitor Map.] *[http://docs.unh.edu/town
 s/CambridgeMassachusettsMapList.htm Old USGS maps of Cambridge area.] {{Greater Boston}} {{Middlesex County, Massachusetts}} {{Massachusetts}} {{New England}} {{Massachusetts cities and mayors of 100,000 population}} [[Category:Cambridge, Massachusetts| ]] [[Category:University towns in the United States]] [[Category:County seats in Massachusetts]] [[Category:Populated places established in 1630]] [[Category:Charles River]] [[Category:Place names of English origin in the United States]] [[af:Cambridge, Massachusetts]] [[ar:\u0643\u0627\u0645\u0628\u0631\u064a\u062f\u062c\u060c \u0645\u0627\u0633\u0627\u062a\u0634\u0648\u0633\u062a\u0633]] [[zh-min-nan:Cambridge, Massachusetts]] [[be:\u0413\u043e\u0440\u0430\u0434 \u041a\u0435\u043c\u0431\u0440\u044b\u0434\u0436, \u041c\u0430\u0441\u0430\u0447\u0443\u0441\u0435\u0442\u0441]] [[be-x-old:\u041a\u0435\u043c\u0431\u0440\u044b\u0434\u0436 (\u041c\u0430\u0441\u0430\u0447\u0443\u0441\u044d\u0442\u0441)]] [[bg:\u041a\u0435\u0439\u043c\u0431\u0440\u0438\u0434\u0436 (\u041c\u0430\u0441\u0430\u0447\u0443\u0437\u0435\u0442\u0441)]] [[br:Cambridge (Massachusetts)]] [[ca:Cambridge (Massachusetts)]] [[cs:Cambridge (Massachusetts)]] [[cy:Cambridge, Massachusetts]] [[da:Cambridge (Massachusetts)]] [[de:Cambridge (Massachusetts)]] [[et:Cambridge (Massachusetts)
 ]] [[es:Cambridge (Massachusetts)]] [[eo:Kembri\u011do (Masa\u0109useco)]] [[eu:Cambridge (Massachusetts)]] [[fa:\u06a9\u0645\u0628\u0631\u06cc\u062c (\u0645\u0627\u0633\u0627\u0686\u0648\u0633\u062a)]] [[fr:Cambridge (Massachusetts)]] [[gd:Cambridge (MA)]] [[ko:\ucf00\uc784\ube0c\ub9ac\uc9c0 (\ub9e4\uc0ac\ucd94\uc138\uce20 \uc8fc)]] [[hy:\u0554\u0565\u0574\u0562\u0580\u056b\u057b (\u0544\u0561\u057d\u0561\u0579\u0578\u0582\u057d\u0565\u0569\u057d)]] [[id:Cambridge, Massachusetts]] [[it:Cambridge (Massachusetts)]] [[he:\u05e7\u05d9\u05d9\u05de\u05d1\u05e8\u05d9\u05d3\u05d2' (\u05de\u05e1\u05e6'\u05d5\u05e1\u05d8\u05e1)]] [[jv:Cambridge, Massachusetts]] [[kk:\u041a\u044d\u043c\u0431\u0440\u0438\u0434\u0436 (\u041c\u0430\u0441\u0441\u0430\u0447\u0443\u0441\u0435\u0442\u0441)]] [[kw:Cambridge, Massachusetts]] [[sw:Cambridge, Massachusetts]] [[ht:Cambridge, Massachusetts]] [[la:Cantabrigia (Massachusetta)]] [[lv:Keimbrid\u017ea]] [[lb:Cambridge (Massachusetts)]] [[hu:Cambridge (Massachusetts)]] [[mr:\u0915\u0947\u0902\u092c\u094d\u0930\u093f\u091c, \u092e\u0945\u0938\u0947\u091a\u094d\u092f\u0941\u0938\u0947\u091f\u094d\u0938]] [[ms:Cambridge, Massachusetts]] [[nl:Cambridge (Massachusetts)]] [[ja:\u30b1\u30f3\u30d6\u30ea\u30c3\u30b8 (\u30de\u30b5\u30c1\u30e5\u30fc\u30bb\u30c3\u30c4\u5dde)]] [[no:Cambridge (Massachusetts)]] [[pl:Cambridge (Massachusetts)]] [[pt:Cambridge (Massachusetts)]] [[ro:Cambri
 dge, Massachusetts]] [[ru:\u041a\u0435\u043c\u0431\u0440\u0438\u0434\u0436 (\u041c\u0430\u0441\u0441\u0430\u0447\u0443\u0441\u0435\u0442\u0441)]] [[scn:Cambridge (Massachusetts), USA]] [[simple:Cambridge, Massachusetts]] [[sk:Cambridge (Massachusetts)]] [[sl:Cambridge, Massachusetts]] [[sr:\u041a\u0435\u043c\u0431\u0440\u0438\u045f (\u041c\u0430\u0441\u0430\u0447\u0443\u0441\u0435\u0442\u0441)]] [[fi:Cambridge (Massachusetts)]] [[sv:Cambridge, Massachusetts]] [[tl:Cambridge, Massachusetts]] [[ta:\u0b95\u0bc7\u0bae\u0bcd\u0baa\u0bbf\u0bb0\u0bbf\u0b9c\u0bcd, \u0bae\u0bbe\u0b9a\u0b9a\u0bc2\u0b9a\u0bc6\u0b9f\u0bcd\u0bb8\u0bcd]] [[th:\u0e40\u0e04\u0e21\u0e1a\u0e23\u0e34\u0e14\u0e08\u0e4c (\u0e23\u0e31\u0e10\u0e41\u0e21\u0e2a\u0e0b\u0e32\u0e0a\u0e39\u0e40\u0e0b\u0e15\u0e2a\u0e4c)]] [[tg:\u041a\u0435\u043c\u0431\u0440\u0438\u04b7 (\u041c\u0430\u0441\u0441\u0430\u0447\u0443\u0441\u0435\u0442\u0441)]] [[tr:Cambridge, Massachusetts]] [[uk:\u041a\u0435\u043c\u0431\u0440\u0438\u0434\u0436 (\u041c\u0430\u0441\u0441\u0430\u0447\u0443\u0441\u0435\u0442\u0441)]] [[vi:Cambridge, Massachusetts]] [[vo:Cambridge (Massachusetts)]] [[war:Cambridge, Massachusetts]] [[yi:\u05e7\u05e2\u05de\u05d1\u05e8\u05d9\u05d3\u05d6\u05e9, \u05de\u05d0\u05e1\u05d0\u05d8\u05e9\u05d5\u05e1\u05e2\u05d8\u05e1]] [[zh:\u5251\u6865 (\u9a6c\u8428\u8bf8\u585e\u5dde)]]
\ No newline at end of file