You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2013/10/21 19:13:23 UTC

svn commit: r1534281 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/util/ core/src/test/org/apache/lucene/index/ core/src/test/org/apache/lucene/util/ highlighter/src/java/org/apache/lucene/sear...

Author: jpountz
Date: Mon Oct 21 17:13:23 2013
New Revision: 1534281

URL: http://svn.apache.org/r1534281
Log:
LUCENE-5274: FastVectorHighlighter now supports highlighting against several indexed fields

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java   (with props)
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMergedIterator.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MergedIterator.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java
    lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
    lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java
    lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldTermStackTest.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Mon Oct 21 17:13:23 2013
@@ -117,6 +117,9 @@ New Features
   compute each suggestion's weight using a javascript expression.
   (Areek Zillur via Mike McCandless)
 
+* LUCENE-5274: FastVectorHighlighter now supports highlighting against several
+  indexed fields. (Nik Everett via Adrien Grand)
+
 Bug Fixes
 
 * LUCENE-4998: Fixed a few places to pass IOContext.READONCE instead

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CoalescedDeletes.java Mon Oct 21 17:13:23 2013
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.lucene.search.Query;
 import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
+import org.apache.lucene.util.MergedIterator;
 
 class CoalescedDeletes {
   final Map<Query,Integer> queries = new HashMap<Query,Integer>();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Mon Oct 21 17:13:23 2013
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHa
 
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MergedIterator;
 
 /**
  * Exposes flex API, merged from flex API of sub-segments.

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java?rev=1534281&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/MergedIterator.java Mon Oct 21 17:13:23 2013
@@ -0,0 +1,160 @@
+package org.apache.lucene.util;
+
+/*
+ * 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.
+ */
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * Provides a merged sorted view from several sorted iterators.
+ * <p>
+ * If built with <code>removeDuplicates</code> set to true and an element
+ * appears in multiple iterators then it is deduplicated, that is this iterator
+ * returns the sorted union of elements.
+ * <p>
+ * If built with <code>removeDuplicates</code> set to false then all elements
+ * in all iterators are returned.
+ * <p>
+ * Caveats:
+ * <ul>
+ *   <li>The behavior is undefined if the iterators are not actually sorted.
+ *   <li>Null elements are unsupported.
+ *   <li>If removeDuplicates is set to true and if a single iterator contains
+ *       duplicates then they will not be deduplicated.
+ *   <li>When elements are deduplicated it is not defined which one is returned.
+ *   <li>If removeDuplicates is set to false then the order in which duplicates
+ *       are returned isn't defined.
+ * </ul>
+ * @lucene.internal
+ */
+public final class MergedIterator<T extends Comparable<T>> implements Iterator<T> {
+  private T current;
+  private final TermMergeQueue<T> queue; 
+  private final SubIterator<T>[] top;
+  private final boolean removeDuplicates;
+  private int numTop;
+
+  @SuppressWarnings({"unchecked","rawtypes"})
+  public MergedIterator(Iterator<T>... iterators) {
+    this(true, iterators);
+  }
+
+  @SuppressWarnings({"unchecked","rawtypes"})
+  public MergedIterator(boolean removeDuplicates, Iterator<T>... iterators) {
+    this.removeDuplicates = removeDuplicates;
+    queue = new TermMergeQueue<T>(iterators.length);
+    top = new SubIterator[iterators.length];
+    int index = 0;
+    for (Iterator<T> iterator : iterators) {
+      if (iterator.hasNext()) {
+        SubIterator<T> sub = new SubIterator<T>();
+        sub.current = iterator.next();
+        sub.iterator = iterator;
+        sub.index = index++;
+        queue.add(sub);
+      }
+    }
+  }
+  
+  @Override
+  public boolean hasNext() {
+    if (queue.size() > 0) {
+      return true;
+    }
+    
+    for (int i = 0; i < numTop; i++) {
+      if (top[i].iterator.hasNext()) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  @Override
+  public T next() {
+    // restore queue
+    pushTop();
+    
+    // gather equal top elements
+    if (queue.size() > 0) {
+      pullTop();
+    } else {
+      current = null;
+    }
+    if (current == null) {
+      throw new NoSuchElementException();
+    }
+    return current;
+  }
+  
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+  
+  private void pullTop() {
+    assert numTop == 0;
+    top[numTop++] = queue.pop();
+    if (removeDuplicates) {
+      // extract all subs from the queue that have the same top element
+      while (queue.size() != 0
+             && queue.top().current.equals(top[0].current)) {
+        top[numTop++] = queue.pop();
+      }
+    }
+    current = top[0].current;
+  }
+  
+  private void pushTop() {
+    // call next() on each top, and put back into queue
+    for (int i = 0; i < numTop; i++) {
+      if (top[i].iterator.hasNext()) {
+        top[i].current = top[i].iterator.next();
+        queue.add(top[i]);
+      } else {
+        // no more elements
+        top[i].current = null;
+      }
+    }
+    numTop = 0;
+  }
+  
+  private static class SubIterator<I extends Comparable<I>> {
+    Iterator<I> iterator;
+    I current;
+    int index;
+  }
+  
+  private static class TermMergeQueue<C extends Comparable<C>> extends PriorityQueue<SubIterator<C>> {
+    TermMergeQueue(int size) {
+      super(size);
+    }
+    
+    @Override
+    protected boolean lessThan(SubIterator<C> a, SubIterator<C> b) {
+      final int cmp = a.current.compareTo(b.current);
+      if (cmp != 0) {
+        return cmp < 0;
+      } else {
+        return a.index < b.index;
+      }
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java Mon Oct 21 17:13:23 2013
@@ -25,6 +25,7 @@ import java.util.Set;
 import java.util.TreeSet;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.MergedIterator;
 import org.apache.lucene.util._TestUtil;
 
 public class TestPrefixCodedTerms extends LuceneTestCase {
@@ -66,15 +67,6 @@ public class TestPrefixCodedTerms extend
     }
     assertFalse(expected.hasNext());
   }
-  
-  @SuppressWarnings("unchecked")
-  public void testMergeEmpty() {
-    Iterator<Term> merged = new MergedIterator<Term>();
-    assertFalse(merged.hasNext());
-
-    merged = new MergedIterator<Term>(new PrefixCodedTerms.Builder().finish().iterator(), new PrefixCodedTerms.Builder().finish().iterator());
-    assertFalse(merged.hasNext());
-  }
 
   @SuppressWarnings("unchecked")
   public void testMergeOne() {

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMergedIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMergedIterator.java?rev=1534281&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMergedIterator.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestMergedIterator.java Mon Oct 21 17:13:23 2013
@@ -0,0 +1,154 @@
+package org.apache.lucene.util;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+
+public class TestMergedIterator extends LuceneTestCase {
+  private static final int REPEATS = 2;
+  private static final int VALS_TO_MERGE = 15000;
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  public void testMergeEmpty() {
+    Iterator<Integer> merged = new MergedIterator<Integer>();
+    assertFalse(merged.hasNext());
+
+    merged = new MergedIterator<Integer>(new ArrayList<Integer>().iterator());
+    assertFalse(merged.hasNext());
+
+    Iterator<Integer>[] itrs = new Iterator[random().nextInt(100)];
+    for (int i = 0; i < itrs.length; i++) {
+      itrs[i] = new ArrayList<Integer>().iterator();
+    }
+    merged = new MergedIterator<Integer>( itrs );
+    assertFalse(merged.hasNext());
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testNoDupsRemoveDups() {
+    testCase(1, 1, true);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testOffItrDupsRemoveDups() {
+    testCase(3, 1, true);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testOnItrDupsRemoveDups() {
+    testCase(1, 3, true);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testOnItrRandomDupsRemoveDups() {
+    testCase(1, -3, true);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testBothDupsRemoveDups() {
+    testCase(3, 3, true);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testBothDupsWithRandomDupsRemoveDups() {
+    testCase(3, -3, true);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testNoDupsKeepDups() {
+    testCase(1, 1, false);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testOffItrDupsKeepDups() {
+    testCase(3, 1, false);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testOnItrDupsKeepDups() {
+    testCase(1, 3, false);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testOnItrRandomDupsKeepDups() {
+    testCase(1, -3, false);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testBothDupsKeepDups() {
+    testCase(3, 3, false);
+  }
+
+  @Repeat(iterations = REPEATS)
+  public void testBothDupsWithRandomDupsKeepDups() {
+    testCase(3, -3, false);
+  }
+
+  private void testCase(int itrsWithVal, int specifiedValsOnItr, boolean removeDups) {
+    // Build a random number of lists
+    List<Integer> expected = new ArrayList<Integer>();
+    Random random = new Random(random().nextLong());
+    int numLists = itrsWithVal + random.nextInt(1000 - itrsWithVal);
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    List<Integer>[] lists = new List[numLists];
+    for (int i = 0; i < numLists; i++) {
+      lists[i] = new ArrayList<Integer>();
+    }
+    int start = random.nextInt(1000000);
+    int end = start + VALS_TO_MERGE / itrsWithVal / Math.abs(specifiedValsOnItr);
+    for (int i = start; i < end; i++) {
+      int maxList = lists.length;
+      int maxValsOnItr = 0;
+      int sumValsOnItr = 0;
+      for (int itrWithVal = 0; itrWithVal < itrsWithVal; itrWithVal++) {
+        int list = random.nextInt(maxList);
+        int valsOnItr = specifiedValsOnItr < 0 ? (1 + random.nextInt(-specifiedValsOnItr)) : specifiedValsOnItr;
+        maxValsOnItr = Math.max(maxValsOnItr, valsOnItr);
+        sumValsOnItr += valsOnItr;
+        for (int valOnItr = 0; valOnItr < valsOnItr; valOnItr++) {
+          lists[list].add(i);
+        }
+        maxList = maxList - 1;
+        ArrayUtil.swap(lists, list, maxList);
+      }
+      int maxCount = removeDups ? maxValsOnItr : sumValsOnItr;
+      for (int count = 0; count < maxCount; count++) {
+        expected.add(i);
+      }
+    }
+    // Now check that they get merged cleanly
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    Iterator<Integer>[] itrs = new Iterator[numLists];
+    for (int i = 0; i < numLists; i++) {
+      itrs[i] = lists[i].iterator();
+    }
+    
+    MergedIterator<Integer> mergedItr = new MergedIterator<Integer>(removeDups, itrs);
+    Iterator<Integer> expectedItr = expected.iterator();
+    while (expectedItr.hasNext()) {
+      assertTrue(mergedItr.hasNext());
+      assertEquals(expectedItr.next(), mergedItr.next()); 
+    }
+    assertFalse(mergedItr.hasNext());
+  }
+}

Modified: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FastVectorHighlighter.java Mon Oct 21 17:13:23 2013
@@ -18,6 +18,8 @@ package org.apache.lucene.search.vectorh
  */
 
 import java.io.IOException;
+import java.util.Iterator;
+import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
@@ -28,7 +30,6 @@ import org.apache.lucene.search.highligh
  *
  */
 public class FastVectorHighlighter {
-
   public static final boolean DEFAULT_PHRASE_HIGHLIGHT = true;
   public static final boolean DEFAULT_FIELD_MATCH = true;
   private final boolean phraseHighlight;
@@ -186,16 +187,71 @@ public class FastVectorHighlighter {
     return fragmentsBuilder.createFragments( reader, docId, fieldName, fieldFragList, maxNumFragments,
         preTags, postTags, encoder );
   }
-  
+
+  /**
+   * Return the best fragments.  Matches are scanned from matchedFields and turned into fragments against
+   * storedField.  The highlighting may not make sense if matchedFields has matches with offsets that don't
+   * correspond features in storedField.  It will outright throw a {@code StringIndexOutOfBoundsException}
+   * if matchedFields produces offsets outside of storedField.  As such it is advisable that all
+   * matchedFields share the same source as storedField or are at least a prefix of it.
+   * 
+   * @param fieldQuery {@link FieldQuery} object
+   * @param reader {@link IndexReader} of the index
+   * @param docId document id to be highlighted
+   * @param storedField field of the document that stores the text
+   * @param matchedFields fields of the document to scan for matches
+   * @param fragCharSize the length (number of chars) of a fragment
+   * @param maxNumFragments maximum number of fragments
+   * @param fragListBuilder {@link FragListBuilder} object
+   * @param fragmentsBuilder {@link FragmentsBuilder} object
+   * @param preTags pre-tags to be used to highlight terms
+   * @param postTags post-tags to be used to highlight terms
+   * @param encoder an encoder that generates encoded text
+   * @return created fragments or null when no fragments created.
+   *         size of the array can be less than maxNumFragments
+   * @throws IOException If there is a low-level I/O error
+   */
+  public final String[] getBestFragments( final FieldQuery fieldQuery, IndexReader reader, int docId,
+      String storedField, Set< String > matchedFields, int fragCharSize, int maxNumFragments,
+      FragListBuilder fragListBuilder, FragmentsBuilder fragmentsBuilder,
+      String[] preTags, String[] postTags, Encoder encoder ) throws IOException {
+    FieldFragList fieldFragList =
+      getFieldFragList( fragListBuilder, fieldQuery, reader, docId, matchedFields, fragCharSize );
+    return fragmentsBuilder.createFragments( reader, docId, storedField, fieldFragList, maxNumFragments,
+        preTags, postTags, encoder );
+  }
+
+  /**
+   * Build a FieldFragList for one field.
+   */
   private FieldFragList getFieldFragList( FragListBuilder fragListBuilder,
       final FieldQuery fieldQuery, IndexReader reader, int docId,
-      String fieldName, int fragCharSize ) throws IOException {
-    FieldTermStack fieldTermStack = new FieldTermStack( reader, docId, fieldName, fieldQuery );
+      String matchedField, int fragCharSize ) throws IOException {
+    FieldTermStack fieldTermStack = new FieldTermStack( reader, docId, matchedField, fieldQuery );
     FieldPhraseList fieldPhraseList = new FieldPhraseList( fieldTermStack, fieldQuery, phraseLimit );
     return fragListBuilder.createFieldFragList( fieldPhraseList, fragCharSize );
   }
 
   /**
+   * Build a FieldFragList for more than one field.
+   */
+  private FieldFragList getFieldFragList( FragListBuilder fragListBuilder,
+      final FieldQuery fieldQuery, IndexReader reader, int docId,
+      Set< String > matchedFields, int fragCharSize ) throws IOException {
+    Iterator< String > matchedFieldsItr = matchedFields.iterator();
+    if ( !matchedFieldsItr.hasNext() ) {
+      throw new IllegalArgumentException( "matchedFields must contain at least on field name." );
+    }
+    FieldPhraseList[] toMerge = new FieldPhraseList[ matchedFields.size() ];
+    int i = 0;
+    while ( matchedFieldsItr.hasNext() ) {
+      FieldTermStack stack = new FieldTermStack( reader, docId, matchedFieldsItr.next(), fieldQuery );
+      toMerge[ i++ ] = new FieldPhraseList( stack, fieldQuery, phraseLimit );
+    } 
+    return fragListBuilder.createFieldFragList( new FieldPhraseList( toMerge ), fragCharSize );
+  }
+
+  /**
    * return whether phraseHighlight or not.
    * 
    * @return whether phraseHighlight or not

Modified: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldPhraseList.java Mon Oct 21 17:13:23 2013
@@ -17,18 +17,23 @@ package org.apache.lucene.search.vectorh
  */
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.lucene.search.vectorhighlight.FieldQuery.QueryPhraseMap;
 import org.apache.lucene.search.vectorhighlight.FieldTermStack.TermInfo;
+import org.apache.lucene.util.MergedIterator;
 
 /**
  * FieldPhraseList has a list of WeightedPhraseInfo that is used by FragListBuilder
  * to create a FieldFragList object.
  */
 public class FieldPhraseList {
-
+  /**
+   * List of non-overlapping WeightedPhraseInfo objects.
+   */
   LinkedList<WeightedPhraseInfo> phraseList = new LinkedList<WeightedPhraseInfo>();
   
   /**
@@ -106,6 +111,55 @@ public class FieldPhraseList {
     }
   }
 
+  /**
+   * Merging constructor.
+   *
+   * @param toMerge FieldPhraseLists to merge to build this one
+   */
+  public FieldPhraseList( FieldPhraseList[] toMerge ) {
+    // Merge all overlapping WeightedPhraseInfos
+    // Step 1.  Sort by startOffset, endOffset, and boost, in that order.
+    @SuppressWarnings( { "rawtypes", "unchecked" } )
+    Iterator< WeightedPhraseInfo >[] allInfos = new Iterator[ toMerge.length ];
+    int index = 0;
+    for ( FieldPhraseList fplToMerge : toMerge ) {
+      allInfos[ index++ ] = fplToMerge.phraseList.iterator();
+    }
+    MergedIterator< WeightedPhraseInfo > itr = new MergedIterator< WeightedPhraseInfo >( false, allInfos );
+    // Step 2.  Walk the sorted list merging infos that overlap
+    phraseList = new LinkedList< WeightedPhraseInfo >();
+    if ( !itr.hasNext() ) {
+      return;
+    }
+    List< WeightedPhraseInfo > work = new ArrayList< WeightedPhraseInfo >();
+    WeightedPhraseInfo first = itr.next();
+    work.add( first );
+    int workEndOffset = first.getEndOffset();
+    while ( itr.hasNext() ) {
+      WeightedPhraseInfo current = itr.next();
+      if ( current.getStartOffset() <= workEndOffset ) {
+        workEndOffset = Math.max( workEndOffset, current.getEndOffset() );
+        work.add( current );
+      } else {
+        if ( work.size() == 1 ) {
+          phraseList.add( work.get( 0 ) );
+          work.set( 0, current );
+        } else {
+          phraseList.add( new WeightedPhraseInfo( work ) );
+          work.clear();
+          work.add( current );
+        }
+        workEndOffset = current.getEndOffset();
+      }
+    }
+    if ( work.size() == 1 ) {
+      phraseList.add( work.get( 0 ) );
+    } else {
+      phraseList.add( new WeightedPhraseInfo( work ) );
+      work.clear();
+    }
+  }
+
   public void addIfNoOverlap( WeightedPhraseInfo wpi ){
     for( WeightedPhraseInfo existWpi : getPhraseList() ){
       if( existWpi.isOffsetOverlap( wpi ) ) {
@@ -121,9 +175,7 @@ public class FieldPhraseList {
   /**
    * Represents the list of term offsets and boost for some text
    */
-  public static class WeightedPhraseInfo {
-
-    private String text;  // unnecessary member, just exists for debugging purpose
+  public static class WeightedPhraseInfo implements Comparable< WeightedPhraseInfo > {
     private List<Toffs> termsOffsets;   // usually termsOffsets.size() == 1,
                             // but if position-gap > 1 and slop > 0 then size() could be greater than 1
     private float boost;  // query boost
@@ -132,10 +184,15 @@ public class FieldPhraseList {
     private ArrayList<TermInfo> termsInfos;
     
     /**
+     * Text of the match, calculated on the fly.  Use for debugging only.
      * @return the text
      */
     public String getText() {
-      return text;
+      StringBuilder text = new StringBuilder();
+      for ( TermInfo ti: termsInfos ) {
+        text.append( ti.getText() );
+      }
+      return text.toString();
     }
 
     /**
@@ -174,15 +231,11 @@ public class FieldPhraseList {
       TermInfo ti = terms.get( 0 );
       termsOffsets.add( new Toffs( ti.getStartOffset(), ti.getEndOffset() ) );
       if( terms.size() == 1 ){
-        text = ti.getText();
         return;
       }
-      StringBuilder sb = new StringBuilder();
-      sb.append( ti.getText() );
       int pos = ti.getPosition();
       for( int i = 1; i < terms.size(); i++ ){
         ti = terms.get( i );
-        sb.append( ti.getText() );
         if( ti.getPosition() - pos == 1 ){
           Toffs to = termsOffsets.get( termsOffsets.size() - 1 );
           to.setEndOffset( ti.getEndOffset() );
@@ -192,7 +245,50 @@ public class FieldPhraseList {
         }
         pos = ti.getPosition();
       }
-      text = sb.toString();
+    }
+
+    /**
+     * Merging constructor.  Note that this just grabs seqnum from the first info.
+     */
+    public WeightedPhraseInfo( Collection< WeightedPhraseInfo > toMerge ) {
+      // Pretty much the same idea as merging FieldPhraseLists:
+      // Step 1.  Sort by startOffset, endOffset
+      //          While we are here merge the boosts and termInfos
+      Iterator< WeightedPhraseInfo > toMergeItr = toMerge.iterator();
+      if ( !toMergeItr.hasNext() ) {
+        throw new IllegalArgumentException( "toMerge must contain at least one WeightedPhraseInfo." );
+      }
+      WeightedPhraseInfo first = toMergeItr.next();
+      @SuppressWarnings( { "rawtypes", "unchecked" } )
+      Iterator< Toffs >[] allToffs = new Iterator[ toMerge.size() ];
+      termsInfos = new ArrayList< TermInfo >();
+      seqnum = first.seqnum;
+      boost = first.boost;
+      allToffs[ 0 ] = first.termsOffsets.iterator();
+      int index = 1;
+      while ( toMergeItr.hasNext() ) {
+        WeightedPhraseInfo info = toMergeItr.next();
+        boost += info.boost;
+        termsInfos.addAll( info.termsInfos );
+        allToffs[ index++ ] = info.termsOffsets.iterator();
+      }
+      // Step 2.  Walk the sorted list merging overlaps
+      MergedIterator< Toffs > itr = new MergedIterator< Toffs >( false, allToffs );
+      termsOffsets = new ArrayList< Toffs >();
+      if ( !itr.hasNext() ) {
+        return;
+      }
+      Toffs work = itr.next();
+      while ( itr.hasNext() ) {
+        Toffs current = itr.next();
+        if ( current.startOffset <= work.endOffset ) {
+          work.endOffset = Math.max( work.endOffset, current.endOffset );
+        } else {
+          termsOffsets.add( work );
+          work = current;
+        }
+      }
+      termsOffsets.add( work );
     }
     
     public int getStartOffset(){
@@ -202,7 +298,7 @@ public class FieldPhraseList {
     public int getEndOffset(){
       return termsOffsets.get( termsOffsets.size() - 1 ).endOffset;
     }
-    
+
     public boolean isOffsetOverlap( WeightedPhraseInfo other ){
       int so = getStartOffset();
       int eo = getEndOffset();
@@ -218,7 +314,7 @@ public class FieldPhraseList {
     @Override
     public String toString(){
       StringBuilder sb = new StringBuilder();
-      sb.append( text ).append( '(' ).append( boost ).append( ")(" );
+      sb.append( getText() ).append( '(' ).append( boost ).append( ")(" );
       for( Toffs to : termsOffsets ){
         sb.append( to );
       }
@@ -233,10 +329,58 @@ public class FieldPhraseList {
       return seqnum;
     }
 
+    @Override
+    public int compareTo( WeightedPhraseInfo other ) {
+      int diff = getStartOffset() - other.getStartOffset();
+      if ( diff != 0 ) {
+        return diff;
+      }
+      diff = getEndOffset() - other.getEndOffset();
+      if ( diff != 0 ) {
+        return diff;
+      }
+      return (int) Math.signum( getBoost() - other.getBoost() );
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + getStartOffset();
+      result = prime * result + getEndOffset();
+      long b = Double.doubleToLongBits( getBoost() );
+      result = prime * result + ( int )( b ^ ( b >>> 32 ) );
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      WeightedPhraseInfo other = (WeightedPhraseInfo) obj;
+      if (getStartOffset() != other.getStartOffset()) {
+        return false;
+      }
+      if (getEndOffset() != other.getEndOffset()) {
+        return false;
+      }
+      if (getBoost() != other.getBoost()) {
+        return false;
+      }
+      return true;
+    }
+
     /**
      * Term offsets (start + end)
      */
-    public static class Toffs {
+    public static class Toffs implements Comparable< Toffs > {
       private int startOffset;
       private int endOffset;
       public Toffs( int startOffset, int endOffset ){
@@ -253,6 +397,42 @@ public class FieldPhraseList {
         return endOffset;
       }
       @Override
+      public int compareTo( Toffs other ) {
+        int diff = getStartOffset() - other.getStartOffset();
+        if ( diff != 0 ) {
+          return diff;
+        }
+        return getEndOffset() - other.getEndOffset();
+      }
+      @Override
+      public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + getStartOffset();
+        result = prime * result + getEndOffset();
+        return result;
+      }
+      @Override
+      public boolean equals(Object obj) {
+        if (this == obj) {
+          return true;
+        }
+        if (obj == null) {
+          return false;
+        }
+        if (getClass() != obj.getClass()) {
+          return false;
+        }
+        Toffs other = (Toffs) obj;
+        if (getStartOffset() != other.getStartOffset()) {
+          return false;
+        }
+        if (getEndOffset() != other.getEndOffset()) {
+          return false;
+        }
+        return true;
+      }
+      @Override
       public String toString(){
         StringBuilder sb = new StringBuilder();
         sb.append( '(' ).append( startOffset ).append( ',' ).append( endOffset ).append( ')' );

Modified: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java Mon Oct 21 17:13:23 2013
@@ -161,7 +161,8 @@ public class FieldTermStack {
   }
   
   /**
-   * Single term with its position/offsets in the document and IDF weight
+   * Single term with its position/offsets in the document and IDF weight.
+   * It is Comparable but considers only position.
    */
   public static class TermInfo implements Comparable<TermInfo>{
 
@@ -198,5 +199,30 @@ public class FieldTermStack {
     public int compareTo( TermInfo o ){
       return ( this.position - o.position );
     }
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + position;
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      TermInfo other = (TermInfo) obj;
+      if (position != other.position) {
+        return false;
+      }
+      return true;
+    }
   }
 }

Modified: lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java Mon Oct 21 17:13:23 2013
@@ -16,10 +16,18 @@ package org.apache.lucene.search.vectorh
  * limitations under the License.
  */
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.AnalyzerWrapper;
+import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.Token;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -33,10 +41,15 @@ import org.apache.lucene.search.BooleanC
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.highlight.DefaultEncoder;
+import org.apache.lucene.search.highlight.Encoder;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
 
 
 public class FastVectorHighlighterTest extends LuceneTestCase {
@@ -287,4 +300,222 @@ public class FastVectorHighlighterTest e
     writer.close();
     dir.close();
   }
+  
+  public void testMatchedFields() throws IOException {
+    // Searching just on the stored field doesn't highlight a stopword
+    matchedFieldsTestCase( false, true, "a match", "a <b>match</b>",
+      clause( "field", "a" ), clause( "field", "match" ) );
+
+    // Even if you add an unqueried matched field that would match it
+    matchedFieldsTestCase( "a match", "a <b>match</b>",
+      clause( "field", "a" ), clause( "field", "match" ) );
+
+    // Nor if you query the field but don't add it as a matched field to the highlighter
+    matchedFieldsTestCase( false, false, "a match", "a <b>match</b>",
+      clause( "field_exact", "a" ), clause( "field", "match" ) );
+
+    // But if you query the field and add it as a matched field to the highlighter then it is highlighted
+    matchedFieldsTestCase( "a match", "<b>a</b> <b>match</b>",
+      clause( "field_exact", "a" ), clause( "field", "match" ) );
+
+    // It is also ok to match just the matched field but get highlighting from the stored field
+    matchedFieldsTestCase( "a match", "<b>a</b> <b>match</b>",
+      clause( "field_exact", "a" ), clause( "field_exact", "match" ) );
+
+    // Boosted matched fields work too
+    matchedFieldsTestCase( "a match", "<b>a</b> <b>match</b>",
+      clause( "field_exact", 5, "a" ), clause( "field", "match" ) );
+
+    // It is also ok if both the stored and the matched field match the term
+    matchedFieldsTestCase( "a match", "a <b>match</b>",
+      clause( "field_exact", "match" ), clause( "field", "match" ) );
+
+    // And the highlighter respects the boosts on matched fields when sorting fragments
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field", "cat" ), clause( "field_exact", 5, "a", "cat" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> <b>cat</b> junk junk junk junk",
+      clause( "field", "cat" ), clause( "field_exact", "a", "cat" ) );
+
+    // The same thing works across three fields as well
+    matchedFieldsTestCase( "cat cat CAT junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field", "cat" ), clause( "field_exact", 200, "a", "cat" ), clause( "field_super_exact", 5, "CAT" ) );
+    matchedFieldsTestCase( "a cat cat junk junk junk junk junk junk junk a CAT junk junk",
+      "junk junk <b>a CAT</b> junk junk",
+      clause( "field", "cat" ), clause( "field_exact", 5, "a", "cat" ), clause( "field_super_exact", 200, "a", "CAT" ) );
+
+    // And across fields with different tokenizers!
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field_exact", 5, "a", "cat" ), clause( "field_characters", "c" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>c</b>at <b>c</b>at junk junk junk junk",
+      clause( "field_exact", "a", "cat" ), clause( "field_characters", "c" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "ca<b>t</b> ca<b>t</b> junk junk junk junk",
+      clause( "field_exact", "a", "cat" ), clause( "field_characters", "t" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> <b>cat</b> junk junk junk junk", // See how the phrases are joined?
+      clause( "field", "cat" ), clause( "field_characters", 5, "c" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field", "cat" ), clause( "field_characters", 5, "a", " ", "c", "a", "t" ) );
+
+    // Phrases and tokens inside one another are joined
+    matchedFieldsTestCase( "cats wow", "<b>cats w</b>ow",
+      clause( "field", "cats" ), clause( "field_tripples", "s w" ) );
+
+    // Everything works pretty well even if you don't require a field match
+    matchedFieldsTestCase( true, false, "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field", "cat" ), clause( "field_characters", 10, "a", " ", "c", "a", "t" ) );
+
+    // Even boosts keep themselves pretty much intact
+    matchedFieldsTestCase( true, false, "a cat cat junk junk junk junk junk junk junk a CAT junk junk",
+      "junk junk <b>a CAT</b> junk junk",
+      clause( "field", "cat" ), clause( "field_exact", 5, "a", "cat" ), clause( "field_super_exact", 200, "a", "CAT" ) );
+    matchedFieldsTestCase( true, false, "cat cat CAT junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field", "cat" ), clause( "field_exact", 200, "a", "cat" ), clause( "field_super_exact", 5, "CAT" ) );
+
+    // Except that all the matched field matches apply even if they aren't mentioned in the query
+    // which can make for some confusing scoring.  This isn't too big a deal, just something you
+    // need to think about when you don't force a field match.
+    matchedFieldsTestCase( true, false, "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> <b>cat</b> junk junk junk junk",
+      clause( "field", "cat" ), clause( "field_characters", 4, "a", " ", "c", "a", "t" ) );
+
+    // It is also cool to match fields that don't have _exactly_ the same text so long as you are careful.
+    // In this case field_sliced is a prefix of field.
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> <b>cat</b> junk junk junk junk", clause( "field_sliced", "cat" ) );
+
+    // Multiple matches add to the score of the segment
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> <b>cat</b> junk junk junk junk",
+      clause( "field", "cat" ), clause( "field_sliced", "cat" ), clause( "field_exact", 2, "a", "cat" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "junk junk <b>a cat</b> junk junk",
+      clause( "field", "cat" ), clause( "field_sliced", "cat" ), clause( "field_exact", 4, "a", "cat" ) );
+
+    // Even fields with tokens on top of one another are ok
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> cat junk junk junk junk",
+      clause( "field_der_red", 2, "der" ), clause( "field_exact", "a", "cat" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> cat junk junk junk junk",
+      clause( "field_der_red", 2, "red" ), clause( "field_exact", "a", "cat" ) );
+    matchedFieldsTestCase( "cat cat junk junk junk junk junk junk junk a cat junk junk",
+      "<b>cat</b> cat junk junk junk junk",
+      clause( "field_der_red", "red" ), clause( "field_der_red", "der" ), clause( "field_exact", "a", "cat" ) );
+  }
+
+  private void matchedFieldsTestCase( String fieldValue, String expected, Query... queryClauses ) throws IOException {
+    matchedFieldsTestCase( true, true, fieldValue, expected, queryClauses );
+  }
+
+  private void matchedFieldsTestCase( boolean useMatchedFields, boolean fieldMatch, String fieldValue, String expected, Query... queryClauses ) throws IOException {
+    Document doc = new Document();
+    FieldType stored = new FieldType( TextField.TYPE_STORED );
+    stored.setStoreTermVectorOffsets( true );
+    stored.setStoreTermVectorPositions( true );
+    stored.setStoreTermVectors( true );
+    stored.freeze();
+    FieldType matched = new FieldType( TextField.TYPE_NOT_STORED );
+    matched.setStoreTermVectorOffsets( true );
+    matched.setStoreTermVectorPositions( true );
+    matched.setStoreTermVectors( true );
+    matched.freeze();
+    doc.add( new Field( "field", fieldValue, stored ) );               // Whitespace tokenized with English stop words
+    doc.add( new Field( "field_exact", fieldValue, matched ) );        // Whitespace tokenized without stop words
+    doc.add( new Field( "field_super_exact", fieldValue, matched ) );  // Whitespace tokenized without toLower
+    doc.add( new Field( "field_characters", fieldValue, matched ) );   // Each letter is a token
+    doc.add( new Field( "field_tripples", fieldValue, matched ) );     // Every three letters is a token
+    doc.add( new Field( "field_sliced", fieldValue.substring( 0,       // Sliced at 10 chars then analyzed just like field
+      Math.min( fieldValue.length() - 1 , 10 ) ), matched ) );
+    doc.add( new Field( "field_der_red", new CannedTokenStream(        // Hacky field containing "der" and "red" at pos = 0
+          token( "der", 1, 0, 3 ),
+          token( "red", 0, 0, 3 )
+        ), matched ) );
+
+    final Map<String, Analyzer> fieldAnalyzers = new TreeMap<String, Analyzer>();
+    fieldAnalyzers.put( "field", new MockAnalyzer( random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET ) );
+    fieldAnalyzers.put( "field_exact", new MockAnalyzer( random() ) );
+    fieldAnalyzers.put( "field_super_exact", new MockAnalyzer( random(), MockTokenizer.WHITESPACE, false ) );
+    fieldAnalyzers.put( "field_characters", new MockAnalyzer( random(), new CharacterRunAutomaton( new RegExp(".").toAutomaton() ), true ) );
+    fieldAnalyzers.put( "field_tripples", new MockAnalyzer( random(), new CharacterRunAutomaton( new RegExp("...").toAutomaton() ), true ) );
+    fieldAnalyzers.put( "field_sliced", fieldAnalyzers.get( "field" ) );
+    fieldAnalyzers.put( "field_der_red", fieldAnalyzers.get( "field" ) );  // This is required even though we provide a token stream
+    Analyzer analyzer = new AnalyzerWrapper() {
+      public Analyzer getWrappedAnalyzer(String fieldName) {
+        return fieldAnalyzers.get( fieldName );
+      }
+    };
+
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter( dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer ) );
+    writer.addDocument( doc );
+
+    FastVectorHighlighter highlighter = new FastVectorHighlighter();
+    FragListBuilder fragListBuilder = new SimpleFragListBuilder();
+    FragmentsBuilder fragmentsBuilder = new ScoreOrderFragmentsBuilder();
+    IndexReader reader = DirectoryReader.open( writer, true );
+    String[] preTags = new String[] { "<b>" };
+    String[] postTags = new String[] { "</b>" };
+    Encoder encoder = new DefaultEncoder();
+    int docId = 0;
+    BooleanQuery query = new BooleanQuery();
+    for ( Query clause : queryClauses ) {
+      query.add( clause, Occur.MUST );
+    }
+    FieldQuery fieldQuery = new FieldQuery( query, reader, true, fieldMatch );
+    String[] bestFragments;
+    if ( useMatchedFields ) {
+      Set< String > matchedFields = new HashSet< String >();
+      matchedFields.add( "field" );
+      matchedFields.add( "field_exact" );
+      matchedFields.add( "field_super_exact" );
+      matchedFields.add( "field_characters" );
+      matchedFields.add( "field_tripples" );
+      matchedFields.add( "field_sliced" );
+      matchedFields.add( "field_der_red" );
+      bestFragments = highlighter.getBestFragments( fieldQuery, reader, docId, "field", matchedFields, 25, 1,
+        fragListBuilder, fragmentsBuilder, preTags, postTags, encoder );
+    } else {
+      bestFragments = highlighter.getBestFragments( fieldQuery, reader, docId, "field", 25, 1,
+        fragListBuilder, fragmentsBuilder, preTags, postTags, encoder );
+    }
+    assertEquals( expected, bestFragments[ 0 ] );
+
+    reader.close();
+    writer.close();
+    dir.close();
+  }
+
+  private Query clause( String field, String... terms ) {
+    return clause( field, 1, terms );
+  }
+
+  private Query clause( String field, float boost, String... terms ) {
+    Query q;
+    if ( terms.length == 1 ) {
+      q = new TermQuery( new Term( field, terms[ 0 ] ) );
+    } else {
+      PhraseQuery pq = new PhraseQuery();
+      for ( String term: terms ) {
+        pq.add( new Term( field, term ) );
+      }
+      q = pq;
+    }
+    q.setBoost( boost );
+    return q;
+  }
+
+  private static Token token( String term, int posInc, int startOffset, int endOffset ) {
+    Token t = new Token( term, startOffset, endOffset );
+    t.setPositionIncrement( posInc );
+    return t;
+  }
 }

Modified: lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java Mon Oct 21 17:13:23 2013
@@ -16,8 +16,14 @@ package org.apache.lucene.search.vectorh
  * limitations under the License.
  */
 
+import java.util.LinkedList;
+
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.vectorhighlight.FieldPhraseList.WeightedPhraseInfo;
+import org.apache.lucene.search.vectorhighlight.FieldPhraseList.WeightedPhraseInfo.Toffs;
+import org.apache.lucene.search.vectorhighlight.FieldTermStack.TermInfo;
+import org.apache.lucene.util._TestUtil;
 
 public class FieldPhraseListTest extends AbstractTestCase {
   
@@ -188,7 +194,7 @@ public class FieldPhraseListTest extends
     assertEquals( 1, fpl.phraseList.size() );
     assertEquals( "sppeeeed(1.0)((88,93))", fpl.phraseList.get( 0 ).toString() );
   }
-  
+
   /* This test shows a big speedup from limiting the number of analyzed phrases in 
    * this bad case for FieldPhraseList */
   /* But it is not reliable as a unit test since it is timing-dependent
@@ -218,4 +224,68 @@ public class FieldPhraseListTest extends
       assertEquals( "a(1.0)((0,1))", fpl.phraseList.get( 0 ).toString() );      
   }
   */
+
+  public void testWeightedPhraseInfoComparisonConsistency() {
+    WeightedPhraseInfo a = newInfo( 0, 0, 1 );
+    WeightedPhraseInfo b = newInfo( 1, 2, 1 );
+    WeightedPhraseInfo c = newInfo( 2, 3, 1 );
+    WeightedPhraseInfo d = newInfo( 0, 0, 1 );
+    WeightedPhraseInfo e = newInfo( 0, 0, 2 );
+
+    assertConsistentEquals( a, a );
+    assertConsistentEquals( b, b );
+    assertConsistentEquals( c, c );
+    assertConsistentEquals( d, d );
+    assertConsistentEquals( e, e );
+    assertConsistentEquals( a, d );
+    assertConsistentLessThan( a, b );
+    assertConsistentLessThan( b, c );
+    assertConsistentLessThan( a, c );
+    assertConsistentLessThan( a, e );
+    assertConsistentLessThan( e, b );
+    assertConsistentLessThan( e, c );
+    assertConsistentLessThan( d, b );
+    assertConsistentLessThan( d, c );
+    assertConsistentLessThan( d, e );
+  }
+
+  public void testToffsComparisonConsistency() {
+    Toffs a = new Toffs( 0, 0 );
+    Toffs b = new Toffs( 1, 2 );
+    Toffs c = new Toffs( 2, 3 );
+    Toffs d = new Toffs( 0, 0 );
+
+    assertConsistentEquals( a, a );
+    assertConsistentEquals( b, b );
+    assertConsistentEquals( c, c );
+    assertConsistentEquals( d, d );
+    assertConsistentEquals( a, d );
+    assertConsistentLessThan( a, b );
+    assertConsistentLessThan( b, c );
+    assertConsistentLessThan( a, c );
+    assertConsistentLessThan( d, b );
+    assertConsistentLessThan( d, c );
+  }
+
+  private WeightedPhraseInfo newInfo( int startOffset, int endOffset, float boost ) {
+    LinkedList< TermInfo > infos = new LinkedList< TermInfo >();
+    infos.add( new TermInfo( _TestUtil.randomUnicodeString( random() ), startOffset, endOffset, 0, 0 ) );
+    return new WeightedPhraseInfo( infos, boost );
+  }
+
+  private < T extends Comparable< T > > void assertConsistentEquals( T a, T b ) {
+    assertEquals( a, b );
+    assertEquals( b, a );
+    assertEquals( a.hashCode(), b.hashCode() );
+    assertEquals( 0, a.compareTo( b ) );
+    assertEquals( 0, b.compareTo( a ) );
+  }
+
+  private < T extends Comparable< T > > void assertConsistentLessThan( T a, T b ) {
+    assertFalse( a.equals( b ) );
+    assertFalse( b.equals( a ) );
+    assertFalse( a.hashCode() == b.hashCode() );
+    assertTrue( a.compareTo( b ) < 0 );
+    assertTrue( b.compareTo( a ) > 0 );
+  }
 }

Modified: lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldTermStackTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldTermStackTest.java?rev=1534281&r1=1534280&r2=1534281&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldTermStackTest.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldTermStackTest.java Mon Oct 21 17:13:23 2013
@@ -20,6 +20,8 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.vectorhighlight.FieldTermStack.TermInfo;
+import org.apache.lucene.util._TestUtil;
 
 public class FieldTermStackTest extends AbstractTestCase {
   
@@ -173,4 +175,37 @@ public class FieldTermStackTest extends 
     assertEquals ("the(195,198,31)", stack.pop().toString());
   }
 
+  public void testTermInfoComparisonConsistency() {
+    TermInfo a = new TermInfo( _TestUtil.randomUnicodeString( random() ), 0, 0, 0, 1 );
+    TermInfo b = new TermInfo( _TestUtil.randomUnicodeString( random() ), 0, 0, 1, 1 );
+    TermInfo c = new TermInfo( _TestUtil.randomUnicodeString( random() ), 0, 0, 2, 1 );
+    TermInfo d = new TermInfo( _TestUtil.randomUnicodeString( random() ), 0, 0, 0, 1 );
+
+    assertConsistentEquals( a, a );
+    assertConsistentEquals( b, b );
+    assertConsistentEquals( c, c );
+    assertConsistentEquals( d, d );
+    assertConsistentEquals( a, d );
+    assertConsistentLessThan( a, b );
+    assertConsistentLessThan( b, c );
+    assertConsistentLessThan( a, c );
+    assertConsistentLessThan( d, b );
+    assertConsistentLessThan( d, c );
+  }
+
+  private < T extends Comparable< T > > void assertConsistentEquals( T a, T b ) {
+    assertEquals( a, b );
+    assertEquals( b, a );
+    assertEquals( a.hashCode(), b.hashCode() );
+    assertEquals( 0, a.compareTo( b ) );
+    assertEquals( 0, b.compareTo( a ) );
+  }
+
+  private < T extends Comparable< T > > void assertConsistentLessThan( T a, T b ) {
+    assertFalse( a.equals( b ) );
+    assertFalse( b.equals( a ) );
+    assertFalse( a.hashCode() == b.hashCode() );
+    assertTrue( a.compareTo( b ) < 0 );
+    assertTrue( b.compareTo( a ) > 0 );
+  }
 }