You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/01/26 05:49:19 UTC

svn commit: r1561451 - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/search/spans/ lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/ lucene/highlighter/src/test...

Author: rmuir
Date: Sun Jan 26 04:49:18 2014
New Revision: 1561451

URL: http://svn.apache.org/r1561451
Log:
LUCENE-5415: add multitermquery support to PostingsHighlighter

Added:
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java   (with props)
    lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestMultiTermHighlighting.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Jan 26 04:49:18 2014
@@ -116,6 +116,9 @@ New Features
   fixes too. More info:
   https://github.com/spatial4j/spatial4j/blob/master/CHANGES.md  (David Smiley)
 
+* LUCENE-5415: Add multitermquery (wildcards,prefix,etc) to PostingsHighlighter.
+  (Mike McCandless, Robert Muir)
+
 Build
 
 * LUCENE-5217: Maven config: get dependencies from Ant+Ivy config; disable
@@ -201,6 +204,9 @@ Bug fixes
   the same Directory to multiple concurrent addIndexes calls (which is
   anyways unusual).  (Robert Muir, Mike McCandless)
 
+* LUCENE-5415: SpanMultiTermQueryWrapper didn't handle its boost in
+  hashcode/equals/tostring/rewrite.  (Robert Muir)
+
 API Changes
 
 * LUCENE-5339: The facet module was simplified/reworked to make the

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java Sun Jan 26 04:49:18 2014
@@ -128,4 +128,9 @@ public class AutomatonQuery extends Mult
     buffer.append(ToStringUtils.boost(getBoost()));
     return buffer.toString();
   }
+  
+  /** Returns the automaton used to create this query */
+  public Automaton getAutomaton() {
+    return automaton;
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyQuery.java Sun Jan 26 04:49:18 2014
@@ -138,6 +138,14 @@ public class FuzzyQuery extends MultiTer
   public int getPrefixLength() {
     return prefixLength;
   }
+  
+  /**
+   * Returns true if transpositions should be treated as a primitive edit operation. 
+   * If this is false, comparisons will implement the classic Levenshtein algorithm.
+   */
+  public boolean getTranspositions() {
+    return transpositions;
+  }
 
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java Sun Jan 26 04:49:18 2014
@@ -100,6 +100,11 @@ public class SpanMultiTermQueryWrapper<Q
   public String getField() {
     return query.getField();
   }
+  
+  /** Returns the wrapped query */
+  public Query getWrappedQuery() {
+    return query;
+  }
 
   @Override
   public String toString(String field) {
@@ -107,6 +112,10 @@ public class SpanMultiTermQueryWrapper<Q
     builder.append("SpanMultiTermQueryWrapper(");
     builder.append(query.toString(field));
     builder.append(")");
+    if (getBoost() != 1F) {
+      builder.append('^');
+      builder.append(getBoost());
+    }
     return builder.toString();
   }
 
@@ -115,22 +124,26 @@ public class SpanMultiTermQueryWrapper<Q
     final Query q = query.rewrite(reader);
     if (!(q instanceof SpanQuery))
       throw new UnsupportedOperationException("You can only use SpanMultiTermQueryWrapper with a suitable SpanRewriteMethod.");
+    q.setBoost(q.getBoost() * getBoost()); // multiply boost
     return q;
   }
   
   @Override
   public int hashCode() {
-    return 31 * query.hashCode();
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + query.hashCode();
+    return result;
   }
 
   @Override
-  @SuppressWarnings({"rawtypes","unchecked"})
   public boolean equals(Object obj) {
     if (this == obj) return true;
-    if (obj == null) return false;
+    if (!super.equals(obj)) return false;
     if (getClass() != obj.getClass()) return false;
-    final SpanMultiTermQueryWrapper other = (SpanMultiTermQueryWrapper) obj;
-    return query.equals(other.query);
+    SpanMultiTermQueryWrapper<?> other = (SpanMultiTermQueryWrapper<?>) obj;
+    if (!query.equals(other.query)) return false;
+    return true;
   }
 
   /** Abstract class that defines how the query is rewritten. */

Added: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java?rev=1561451&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java (added)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/MultiTermHighlighting.java Sun Jan 26 04:49:18 2014
@@ -0,0 +1,284 @@
+package org.apache.lucene.search.postingshighlight;
+
+/*
+ * 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.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.DocsAndPositionsEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.AutomatonQuery;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+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.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.BasicAutomata;
+import org.apache.lucene.util.automaton.BasicOperations;
+import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.LevenshteinAutomata;
+
+/**
+ * Support for highlighting multiterm queries in PostingsHighlighter.
+ */
+class MultiTermHighlighting {
+  
+  /** 
+   * Extracts all MultiTermQueries for {@code field}, and returns equivalent 
+   * automata that will match terms.
+   */
+  static CharacterRunAutomaton[] extractAutomata(Query query, String field) {
+    List<CharacterRunAutomaton> list = new ArrayList<>();
+    if (query instanceof BooleanQuery) {
+      BooleanClause clauses[] = ((BooleanQuery) query).getClauses();
+      for (BooleanClause clause : clauses) {
+        if (!clause.isProhibited()) {
+          list.addAll(Arrays.asList(extractAutomata(clause.getQuery(), field)));
+        }
+      }
+    } else if (query instanceof DisjunctionMaxQuery) {
+      for (Query sub : ((DisjunctionMaxQuery) query).getDisjuncts()) {
+        list.addAll(Arrays.asList(extractAutomata(sub, field)));
+      }
+    } else if (query instanceof SpanOrQuery) {
+      for (Query sub : ((SpanOrQuery) query).getClauses()) {
+        list.addAll(Arrays.asList(extractAutomata(sub, field)));
+      }
+    } else if (query instanceof SpanNearQuery) {
+      for (Query sub : ((SpanNearQuery) query).getClauses()) {
+        list.addAll(Arrays.asList(extractAutomata(sub, field)));
+      }
+    } else if (query instanceof SpanNotQuery) {
+      list.addAll(Arrays.asList(extractAutomata(((SpanNotQuery) query).getInclude(), field)));
+    } else if (query instanceof SpanPositionCheckQuery) {
+      list.addAll(Arrays.asList(extractAutomata(((SpanPositionCheckQuery) query).getMatch(), field)));
+    } else if (query instanceof SpanMultiTermQueryWrapper) {
+      list.addAll(Arrays.asList(extractAutomata(((SpanMultiTermQueryWrapper<?>) query).getWrappedQuery(), field)));
+    } 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(BasicOperations.concatenate(BasicAutomata.makeString(prefix.text()), 
+                                                                       BasicAutomata.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());
+        Automaton automaton = builder.toAutomaton(fq.getMaxEdits());
+        if (prefixLength > 0) {
+          Automaton prefix = BasicAutomata.makeString(UnicodeUtil.newString(termText, 0, prefixLength));
+          automaton = BasicOperations.concatenate(prefix, automaton);
+        }
+        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();
+        final Comparator<CharsRef> comparator = CharsRef.getUTF16SortedAsUTF8Comparator();
+        
+        // this is *not* an automaton, but its very simple
+        list.add(new CharacterRunAutomaton(BasicAutomata.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>
+   */
+  static DocsAndPositionsEnum getDocsEnum(final TokenStream ts, final CharacterRunAutomaton[] matchers) throws IOException {
+    final CharTermAttribute charTermAtt = ts.addAttribute(CharTermAttribute.class);
+    final OffsetAttribute offsetAtt = ts.addAttribute(OffsetAttribute.class);
+    ts.reset();
+    
+    // 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 PH terminate based on offset...
+    
+    return new DocsAndPositionsEnum() {
+      int currentDoc = -1;
+      int currentMatch = -1;
+      int currentStartOffset = -1;
+      int currentEndOffset = -1;
+      TokenStream stream = ts;
+      
+      final BytesRef matchDescriptions[] = new BytesRef[matchers.length];
+      
+      @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();
+          stream.close();
+          stream = null;
+        }
+        // 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;
+      }
+    };
+  }
+}

Modified: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java Sun Jan 26 04:49:18 2014
@@ -30,6 +30,7 @@ import java.util.PriorityQueue;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -50,6 +51,7 @@ import org.apache.lucene.search.TopDocs;
 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;
 
 /**
  * Simple highlighter that does not analyze fields nor use
@@ -64,6 +66,14 @@ import org.apache.lucene.util.UnicodeUti
  * 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 subclassing this highlighter, 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.
+ *   <li>{@link #getIndexAnalyzer(String)}: Enable highlighting of MultiTermQuerys such as {@code WildcardQuery}.
+ * </ul>
+ * <p>
  * <b>WARNING</b>: The code is very new and probably still has some exciting bugs!
  * <p>
  * Example usage:
@@ -335,9 +345,9 @@ public class PostingsHighlighter {
       throw new IllegalArgumentException("invalid number of maxPassagesIn");
     }
     final IndexReader reader = searcher.getIndexReader();
-    query = rewrite(query);
+    Query rewritten = rewrite(query);
     SortedSet<Term> queryTerms = new TreeSet<Term>();
-    query.extractTerms(queryTerms);
+    rewritten.extractTerms(queryTerms);
 
     IndexReaderContext readerContext = reader.getContext();
     List<AtomicReaderContext> leaves = readerContext.leaves();
@@ -389,7 +399,7 @@ public class PostingsHighlighter {
       for(Term term : fieldTerms) {
         terms[termUpto++] = term.bytes();
       }
-      Map<Integer,Object> fieldHighlights = highlightField(field, contents[i], getBreakIterator(field), terms, docids, leaves, numPassages);
+      Map<Integer,Object> fieldHighlights = highlightField(field, contents[i], getBreakIterator(field), terms, docids, leaves, numPassages, query);
         
       Object[] result = new Object[docids.length];
       for (int j = 0; j < docidsIn.length; j++) {
@@ -432,8 +442,18 @@ public class PostingsHighlighter {
   protected char getMultiValuedSeparator(String field) {
     return ' ';
   }
+  
+  /** 
+   * Returns the analyzer originally used to index the content for {@code field}.
+   * <p>
+   * This is used to highlight some MultiTermQueries.
+   * @return Analyzer or null (the default, meaning no special multi-term processing)
+   */
+  protected Analyzer getIndexAnalyzer(String field) {
+    return null;
+  }
     
-  private Map<Integer,Object> highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List<AtomicReaderContext> leaves, int maxPassages) throws IOException {  
+  private Map<Integer,Object> highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List<AtomicReaderContext> leaves, int maxPassages, Query query) throws IOException {  
     Map<Integer,Object> highlights = new HashMap<Integer,Object>();
     
     // reuse in the real sense... for docs in same segment we just advance our old enum
@@ -445,6 +465,21 @@ public class PostingsHighlighter {
     if (fieldFormatter == null) {
       throw new NullPointerException("PassageFormatter cannot be null");
     }
+    
+    // check if we should do any multitermprocessing
+    Analyzer analyzer = getIndexAnalyzer(field);
+    CharacterRunAutomaton automata[] = new CharacterRunAutomaton[0];
+    if (analyzer != null) {
+      automata = MultiTermHighlighting.extractAutomata(query, field);
+    }
+    
+    final BytesRef allTerms[];
+    if (automata.length > 0) {
+      allTerms = new BytesRef[terms.length + 1];
+      System.arraycopy(terms, 0, allTerms, 0, terms.length);
+    } else {
+      allTerms = terms;
+    }
 
     for (int i = 0; i < docids.length; i++) {
       String content = contents[i];
@@ -462,9 +497,14 @@ public class PostingsHighlighter {
       }
       if (leaf != lastLeaf) {
         termsEnum = t.iterator(null);
-        postings = new DocsAndPositionsEnum[terms.length];
+        postings = new DocsAndPositionsEnum[allTerms.length];
+      }
+      if (automata.length > 0) {
+        DocsAndPositionsEnum dp = MultiTermHighlighting.getDocsEnum(analyzer.tokenStream(field, content), automata);
+        dp.advance(doc - subContext.docBase);
+        postings[terms.length] = dp;
       }
-      Passage passages[] = highlightDoc(field, terms, content.length(), bi, doc - subContext.docBase, termsEnum, postings, maxPassages);
+      Passage passages[] = highlightDoc(field, allTerms, content.length(), bi, doc - subContext.docBase, termsEnum, postings, maxPassages);
       if (passages.length == 0) {
         passages = getEmptyHighlight(field, bi, maxPassages);
       }
@@ -593,7 +633,13 @@ public class PostingsHighlighter {
       int tf = 0;
       while (true) {
         tf++;
-        current.addMatch(start, end, terms[off.id]);
+        BytesRef term = terms[off.id];
+        if (term == null) {
+          // multitermquery match, pull from payload
+          term = off.dp.getPayload();
+          assert term != null;
+        }
+        current.addMatch(start, end, term);
         if (off.pos == dp.freq()) {
           break; // removed from pq
         } else {

Added: lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestMultiTermHighlighting.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestMultiTermHighlighting.java?rev=1561451&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestMultiTermHighlighting.java (added)
+++ lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestMultiTermHighlighting.java Sun Jan 26 04:49:18 2014
@@ -0,0 +1,797 @@
+package org.apache.lucene.search.postingshighlight;
+
+/*
+ * 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 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.TextField;
+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.index.FieldInfo.IndexOptions;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+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.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+import org.apache.lucene.search.Sort;
+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.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+
+/** 
+ * Some tests that override {@link PostingsHighlighter#getIndexAnalyzer} to
+ * highlight wilcard, fuzzy, etc queries.
+ */
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
+public class TestMultiTermHighlighting extends LuceneTestCase {
+  
+  public void testWildcards() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    Query query = new WildcardQuery(new Term("body", "te*"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(new WildcardQuery(new Term("bogus", "te*")), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  public void testOnePrefix() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    Query query = new PrefixQuery(new Term("body", "te"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(new PrefixQuery(new Term("bogus", "te")), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  public void testOneRegexp() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    Query query = new RegexpQuery(new Term("body", "te.*"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(new RegexpQuery(new Term("bogus", "te.*")), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  public void testOneFuzzy() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    Query query = new FuzzyQuery(new Term("body", "tets"), 1);
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(new FuzzyQuery(new Term("bogus", "tets"), 1), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  public void testRanges() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    Query query = TermRangeQuery.newStringRange("body", "ta", "tf", true, true);
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, searcher, 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, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, searcher, 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, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, searcher, 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, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(TermRangeQuery.newStringRange("body", "test", "tf", false, true), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(TermRangeQuery.newStringRange("body", "ta", "test", true, false), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, 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();
+    bq.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    bq.add(TermRangeQuery.newStringRange("bogus", "ta", "tf", true, true), BooleanClause.Occur.SHOULD);
+    topDocs = searcher.search(bq, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", bq, searcher, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  public void testWildcardInBoolean() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    BooleanQuery query = new BooleanQuery();
+    query.add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD);
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    query.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD);
+    query.add(new WildcardQuery(new Term("bogus", "te*")), BooleanClause.Occur.MUST_NOT);
+    topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    snippets = highlighter.highlight("body", query, searcher, topDocs);
+    assertEquals(2, snippets.length);
+    assertEquals("This is a test.", snippets[0]);
+    assertEquals("Test a one sentence document.", snippets[1]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  public void testWildcardInDisjunctionMax() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    DisjunctionMaxQuery query = new DisjunctionMaxQuery(0);
+    query.add(new WildcardQuery(new Term("body", "te*")));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    dir.close();
+  }
+  
+  public void testSpanWildcard() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    Query query = new SpanMultiTermQueryWrapper<WildcardQuery>(new WildcardQuery(new Term("body", "te*")));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    dir.close();
+  }
+  
+  public void testSpanOr() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    SpanQuery childQuery = new SpanMultiTermQueryWrapper<WildcardQuery>(new WildcardQuery(new Term("body", "te*")));
+    Query query = new SpanOrQuery(new SpanQuery[] { childQuery });
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    dir.close();
+  }
+  
+  public void testSpanNear() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    SpanQuery childQuery = new SpanMultiTermQueryWrapper<WildcardQuery>(new WildcardQuery(new Term("body", "te*")));
+    Query query = new SpanNearQuery(new SpanQuery[] { childQuery }, 0, true);
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    dir.close();
+  }
+  
+  public void testSpanNot() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    SpanQuery include = new SpanMultiTermQueryWrapper<WildcardQuery>(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, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    dir.close();
+  }
+  
+  public void testSpanPositionCheck() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    SpanQuery childQuery = new SpanMultiTermQueryWrapper<WildcardQuery>(new WildcardQuery(new Term("body", "te*")));
+    Query query = new SpanFirstQuery(childQuery, 1000000);
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(2, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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();
+    dir.close();
+  }
+  
+  /** Runs a query with two MTQs and confirms the formatter
+   *  can tell which query matched which hit. */
+  public void testWhichMTQMatched() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    final Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    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);
+    PostingsHighlighter highlighter = new PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+    };
+    BooleanQuery query = new BooleanQuery();
+    query.add(new WildcardQuery(new Term("body", "te*")), BooleanClause.Occur.SHOULD);
+    query.add(new WildcardQuery(new Term("body", "one")), BooleanClause.Occur.SHOULD);
+    query.add(new WildcardQuery(new Term("body", "se*")), BooleanClause.Occur.SHOULD);
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, 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 PostingsHighlighter() {
+      @Override
+      protected Analyzer getIndexAnalyzer(String field) {
+        return analyzer;
+      }
+      
+      @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, searcher, 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();
+    dir.close();
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/highlight/PostingsSolrHighlighter.java Sun Jan 26 04:49:18 2014
@@ -24,6 +24,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.postingshighlight.DefaultPassageFormatter;
@@ -68,6 +69,7 @@ import org.apache.solr.util.plugin.Plugi
  *       &lt;str name="hl.bs.type"&gt;SENTENCE&lt;/str&gt;
  *       &lt;int name="hl.maxAnalyzedChars"&gt;10000&lt;/int&gt;
  *       &lt;str name="hl.multiValuedSeparatorChar"&gt; &lt;/str&gt;
+ *       &lt;bool name="hl.highlightMultiTerm"&gt;false&lt;/bool&gt;
  *     &lt;/lst&gt;
  *   &lt;/requestHandler&gt;
  * </pre>
@@ -98,6 +100,7 @@ import org.apache.solr.util.plugin.Plugi
  *    <li>hl.bs.variant (string) specifies country code for BreakIterator. default is empty string (root locale)
  *    <li>hl.maxAnalyzedChars specifies how many characters at most will be processed in a document.
  *    <li>hl.multiValuedSeparatorChar specifies the logical separator between values for multi-valued fields.
+ *    <li>hl.highlightMultiTerm enables highlighting for range/wildcard/fuzzy/prefix queries.
  *        NOTE: currently hl.maxAnalyzedChars cannot yet be specified per-field
  *  </ul>
  *  
@@ -132,6 +135,8 @@ public class PostingsSolrHighlighter ext
         maxPassages[i] = params.getFieldInt(fieldNames[i], HighlightParams.SNIPPETS, 1);
       }
       
+      final IndexSchema schema = req.getSchema();
+      
       PostingsHighlighter highlighter = new PostingsHighlighter(maxLength) {
         @Override
         protected Passage[] getEmptyHighlight(String fieldName, BreakIterator bi, int maxPassages) {
@@ -178,6 +183,15 @@ public class PostingsSolrHighlighter ext
           }
           return sep.charAt(0);
         }
+
+        @Override
+        protected Analyzer getIndexAnalyzer(String field) {
+          if (params.getFieldBool(field, HighlightParams.HIGHLIGHT_MULTI_TERM, false)) {
+            return schema.getAnalyzer();
+          } else {
+            return null;
+          }
+        }
       };
       
       Map<String,String[]> snippets = highlighter.highlightFields(fieldNames, query, searcher, docIDs, maxPassages);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java?rev=1561451&r1=1561450&r2=1561451&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/highlight/TestPostingsSolrHighlighter.java Sun Jan 26 04:49:18 2014
@@ -155,4 +155,12 @@ public class TestPostingsSolrHighlighter
         req("q", "text:document", "sort", "id asc", "hl", "true", "hl.encoder", "html"),
         "//lst[@name='highlighting']/lst[@name='103']/arr[@name='text']/str='<em>Document</em>&#32;one&#32;has&#32;a&#32;first&#32;&lt;i&gt;sentence&lt;&#x2F;i&gt;&#46;'");
   }
+  
+  public void testWildcard() {
+    assertQ("simplest test", 
+        req("q", "text:doc*ment", "sort", "id asc", "hl", "true", "hl.highlightMultiTerm", "true"),
+        "count(//lst[@name='highlighting']/*)=2",
+        "//lst[@name='highlighting']/lst[@name='101']/arr[@name='text']/str='<em>document</em> one'",
+        "//lst[@name='highlighting']/lst[@name='102']/arr[@name='text']/str='second <em>document</em>'");
+  }
 }