You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/05/28 10:31:46 UTC

svn commit: r1682170 [1/2] - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/util/fst/ lucene/suggest/ lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ lucene/suggest/src/test/org/apac...

Author: mikemccand
Date: Thu May 28 08:31:45 2015
New Revision: 1682170

URL: http://svn.apache.org/r1682170
Log:
LUCENE-6459: add common suggest API for document based NRT suggester

Added:
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
      - copied unchanged from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestPrefixCompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestRegexCompletionQuery.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestRegexCompletionQuery.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
      - copied, changed from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestSuggestField.java
Removed:
    lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/SuggestFieldTest.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
    lucene/dev/branches/branch_5x/lucene/suggest/   (props changed)
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
    lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Thu May 28 08:31:45 2015
@@ -12,6 +12,12 @@ New Features
   can be used with getMultiValueSeparator render whole field
   values.  (Luca Cavanna via Robert Muir)
 
+* LUCENE-6459: Add common suggest API that mirrors Lucene's
+  Query/IndexSearcher APIs for Document based suggester.
+  Adds PrefixCompletionQuery, RegexCompletionQuery,
+  FuzzyCompletionQuery and ContextQuery.
+  (Areek Zillur via Mike McCandless)
+
 Bug fixes
 
 * LUCENE-6500: ParallelCompositeReader did not always call

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Thu May 28 08:31:45 2015
@@ -251,17 +251,29 @@ public final class Util {
     public FST.Arc<T> arc;
     public T cost;
     public final IntsRefBuilder input;
+    public final float boost;
+    public final CharSequence context;
 
     /** Sole constructor */
     public FSTPath(T cost, FST.Arc<T> arc, IntsRefBuilder input) {
+      this(cost, arc, input, 0, null);
+    }
+
+    public FSTPath(T cost, FST.Arc<T> arc, IntsRefBuilder input, float boost, CharSequence context) {
       this.arc = new FST.Arc<T>().copyFrom(arc);
       this.cost = cost;
       this.input = input;
+      this.boost = boost;
+      this.context = context;
+    }
+
+    public FSTPath<T> newPath(T cost, IntsRefBuilder input) {
+      return new FSTPath<>(cost, this.arc, input, this.boost, this.context);
     }
 
     @Override
     public String toString() {
-      return "input=" + input + " cost=" + cost;
+      return "input=" + input + " cost=" + cost + "context=" + context + "boost=" + boost;
     }
   }
 
@@ -307,13 +319,18 @@ public final class Util {
      * @param comparator the comparator to select the top N
      */
     public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator) {
+      this(fst, topN, maxQueueDepth, comparator, new TieBreakByInputComparator<>(comparator));
+    }
+
+    public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator,
+                        Comparator<FSTPath<T>> pathComparator) {
       this.fst = fst;
       this.bytesReader = fst.getBytesReader();
       this.topN = topN;
       this.maxQueueDepth = maxQueueDepth;
       this.comparator = comparator;
 
-      queue = new TreeSet<>(new TieBreakByInputComparator<>(comparator));
+      queue = new TreeSet<>(pathComparator);
     }
 
     // If back plus this arc is competitive then add to queue:
@@ -354,25 +371,29 @@ public final class Util {
       IntsRefBuilder newInput = new IntsRefBuilder();
       newInput.copyInts(path.input.get());
       newInput.append(path.arc.label);
-      final FSTPath<T> newPath = new FSTPath<>(cost, path.arc, newInput);
 
-      queue.add(newPath);
+      queue.add(path.newPath(cost, newInput));
 
       if (queue.size() == maxQueueDepth+1) {
         queue.pollLast();
       }
     }
 
+    public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input) throws IOException {
+      addStartPaths(node, startOutput, allowEmptyString, input, 0, null);
+    }
+
     /** Adds all leaving arcs, including 'finished' arc, if
      *  the node is final, from this node into the queue.  */
-    public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input) throws IOException {
+    public void addStartPaths(FST.Arc<T> node, T startOutput, boolean allowEmptyString, IntsRefBuilder input,
+                              float boost, CharSequence context) throws IOException {
 
       // De-dup NO_OUTPUT since it must be a singleton:
       if (startOutput.equals(fst.outputs.getNoOutput())) {
         startOutput = fst.outputs.getNoOutput();
       }
 
-      FSTPath<T> path = new FSTPath<>(startOutput, node, input);
+      FSTPath<T> path = new FSTPath<>(startOutput, node, input, boost, context);
       fst.readFirstTargetArc(node, path.arc, bytesReader);
 
       //System.out.println("add start paths");
@@ -493,10 +514,10 @@ public final class Util {
           if (path.arc.label == FST.END_LABEL) {
             // Add final output:
             //System.out.println("    done!: " + path);
-            T finalOutput = fst.outputs.add(path.cost, path.arc.output);
-            if (acceptResult(path.input.get(), finalOutput)) {
+            path.cost = fst.outputs.add(path.cost, path.arc.output);
+            if (acceptResult(path)) {
               //System.out.println("    add result: " + path);
-              results.add(new Result<>(path.input.get(), finalOutput));
+              results.add(new Result<>(path.input.get(), path.cost));
             } else {
               rejectCount++;
             }
@@ -510,6 +531,10 @@ public final class Util {
       return new TopResults<>(rejectCount + topN <= maxQueueDepth, results);
     }
 
+    protected boolean acceptResult(FSTPath<T> path) {
+      return acceptResult(path.input.get(), path.cost);
+    }
+
     protected boolean acceptResult(IntsRef input, T output) {
       return true;
     }

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java Thu May 28 08:31:45 2015
@@ -17,22 +17,10 @@ package org.apache.lucene.search.suggest
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.AnalyzerWrapper;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.TokenStreamToAutomaton;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.Operations;
-import org.apache.lucene.util.automaton.Transition;
 
 /**
  * Wraps an {@link org.apache.lucene.analysis.Analyzer}
@@ -40,15 +28,15 @@ import org.apache.lucene.util.automaton.
  * (e.g. preserving token separators, preserving position increments while converting
  * a token stream to an automaton)
  * <p>
- * Can be used to index {@link SuggestField}
- * and as a query analyzer to {@link SuggestIndexSearcher}
+ * Can be used to index {@link SuggestField} and {@link ContextSuggestField}
+ * and as a query analyzer to {@link PrefixCompletionQuery} amd {@link FuzzyCompletionQuery}
  * <p>
- * NOTE: In most cases, index and query analyzer should have same values for {@link #preservePositionIncrements}
- * and {@link #preserveSep}
+ * NOTE: In most cases, index and query analyzer should have same values for {@link #preservePositionIncrements()}
+ * and {@link #preserveSep()}
  *
  * @lucene.experimental
  */
-public class CompletionAnalyzer extends AnalyzerWrapper {
+public final class CompletionAnalyzer extends AnalyzerWrapper {
 
   /**
    * Represents the separation between tokens, if
@@ -64,7 +52,7 @@ public class CompletionAnalyzer extends
    */
   final static int HOLE_CHARACTER = TokenStreamToAutomaton.HOLE;
 
-  final static int DEFAULT_MAX_GRAPH_EXPANSIONS = -1;
+  final static int DEFAULT_MAX_GRAPH_EXPANSIONS = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
   final static boolean DEFAULT_PRESERVE_SEP = true;
   final static boolean DEFAULT_PRESERVE_POSITION_INCREMENTS = true;
 
@@ -133,6 +121,22 @@ public class CompletionAnalyzer extends
     this(analyzer, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, maxGraphExpansions);
   }
 
+  /**
+   * Returns true if separation between tokens are preserved when converting
+   * the token stream to an automaton
+   */
+  public boolean preserveSep() {
+    return preserveSep;
+  }
+
+  /**
+   * Returns true if position increments are preserved when converting
+   * the token stream to an automaton
+   */
+  public boolean preservePositionIncrements() {
+    return preservePositionIncrements;
+  }
+
   @Override
   protected Analyzer getWrappedAnalyzer(String fieldName) {
     return analyzer;
@@ -141,33 +145,7 @@ public class CompletionAnalyzer extends
   @Override
   protected TokenStreamComponents wrapComponents(String fieldName, TokenStreamComponents components) {
     CompletionTokenStream tokenStream = new CompletionTokenStream(components.getTokenStream(),
-        preserveSep, preservePositionIncrements, SEP_LABEL, maxGraphExpansions);
+        preserveSep, preservePositionIncrements, maxGraphExpansions);
     return new TokenStreamComponents(components.getTokenizer(), tokenStream);
   }
-
-  /**
-   * Converts <code>key</code> to an automaton using
-   * {@link #preservePositionIncrements}, {@link #preserveSep}
-   * and {@link #maxGraphExpansions}
-   */
-  public Automaton toAutomaton(String field, CharSequence key) throws IOException {
-    for (int i = 0; i < key.length(); i++) {
-      switch (key.charAt(i)) {
-        case HOLE_CHARACTER:
-          throw new IllegalArgumentException("lookup key cannot contain HOLE character U+001E; this character is reserved");
-        case SEP_LABEL:
-          throw new IllegalArgumentException("lookup key cannot contain unit separator character U+001F; this character is reserved");
-        default:
-          break;
-      }
-    }
-
-    try (TokenStream tokenStream = analyzer.tokenStream(field, key.toString())) {
-      try(CompletionTokenStream stream = new CompletionTokenStream(tokenStream,
-          preserveSep, preservePositionIncrements, SEP_LABEL, maxGraphExpansions)) {
-        return stream.toAutomaton(tokenStream);
-      }
-    }
-  }
-
 }

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java Thu May 28 08:31:45 2015
@@ -58,7 +58,7 @@ import static org.apache.lucene.search.s
 final class CompletionFieldsConsumer extends FieldsConsumer {
 
   private final String delegatePostingsFormatName;
-  private final Map<String, Long> seenFields = new HashMap<>();
+  private final Map<String, CompletionMetaData> seenFields = new HashMap<>();
   private final SegmentWriteState state;
   private IndexOutput dictOut;
   private FieldsConsumer delegateFieldsConsumer;
@@ -98,7 +98,10 @@ final class CompletionFieldsConsumer ext
       // store lookup, if needed
       long filePointer = dictOut.getFilePointer();
       if (termWriter.finish(dictOut)) {
-        seenFields.put(field, filePointer);
+        seenFields.put(field, new CompletionMetaData(filePointer,
+            termWriter.minWeight,
+            termWriter.maxWeight,
+            termWriter.type));
       }
     }
   }
@@ -124,10 +127,14 @@ final class CompletionFieldsConsumer ext
       // write # of seen fields
       indexOut.writeVInt(seenFields.size());
       // write field numbers and dictOut offsets
-      for (Map.Entry<String, Long> seenField : seenFields.entrySet()) {
+      for (Map.Entry<String, CompletionMetaData> seenField : seenFields.entrySet()) {
         FieldInfo fieldInfo = state.fieldInfos.fieldInfo(seenField.getKey());
         indexOut.writeVInt(fieldInfo.number);
-        indexOut.writeVLong(seenField.getValue());
+        CompletionMetaData metaData = seenField.getValue();
+        indexOut.writeVLong(metaData.filePointer);
+        indexOut.writeVLong(metaData.minWeight);
+        indexOut.writeVLong(metaData.maxWeight);
+        indexOut.writeByte(metaData.type);
       }
       CodecUtil.writeFooter(indexOut);
       CodecUtil.writeFooter(dictOut);
@@ -140,17 +147,36 @@ final class CompletionFieldsConsumer ext
     }
   }
 
+  private static class CompletionMetaData {
+    private final long filePointer;
+    private final long minWeight;
+    private final long maxWeight;
+    private final byte type;
+
+    private CompletionMetaData(long filePointer, long minWeight, long maxWeight, byte type) {
+      this.filePointer = filePointer;
+      this.minWeight = minWeight;
+      this.maxWeight = maxWeight;
+      this.type = type;
+    }
+  }
+
   // builds an FST based on the terms written
   private static class CompletionTermWriter {
 
     private PostingsEnum postingsEnum = null;
     private int docCount = 0;
+    private long maxWeight = 0;
+    private long minWeight = Long.MAX_VALUE;
+    private byte type;
+    private boolean first;
 
     private final BytesRefBuilder scratch = new BytesRefBuilder();
     private final NRTSuggesterBuilder builder;
 
     public CompletionTermWriter() {
       builder = new NRTSuggesterBuilder();
+      first = true;
     }
 
     /**
@@ -160,6 +186,9 @@ final class CompletionFieldsConsumer ext
     public boolean finish(IndexOutput output) throws IOException {
       boolean stored = builder.store(output);
       assert stored || docCount == 0 : "the FST is null but docCount is != 0 actual value: [" + docCount + "]";
+      if (docCount == 0) {
+        minWeight = 0;
+      }
       return stored;
     }
 
@@ -181,7 +210,17 @@ final class CompletionFieldsConsumer ext
           scratch.grow(len);
           scratch.setLength(len);
           input.readBytes(scratch.bytes(), 0, scratch.length());
-          builder.addEntry(docID, scratch.get(), input.readVLong() - 1);
+          long weight = input.readVInt() - 1;
+          maxWeight = Math.max(maxWeight, weight);
+          minWeight = Math.min(minWeight, weight);
+          byte type = input.readByte();
+          if (first) {
+            this.type = type;
+            first = false;
+          } else if (this.type != type) {
+            throw new IllegalArgumentException("single field name has mixed types");
+          }
+          builder.addEntry(docID, scratch.get(), weight);
         }
         docFreq++;
         docCount = Math.max(docCount, docFreq + 1);

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java Thu May 28 08:31:45 2015
@@ -30,7 +30,6 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
@@ -98,9 +97,12 @@ final class CompletionFieldsProducer ext
       for (int i = 0; i < numFields; i++) {
         int fieldNumber = index.readVInt();
         long offset = index.readVLong();
+        long minWeight = index.readVLong();
+        long maxWeight = index.readVLong();
+        byte type = index.readByte();
         FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldNumber);
         // we don't load the FST yet
-        readers.put(fieldInfo.name, new CompletionsTermsReader(offset));
+        readers.put(fieldInfo.name, new CompletionsTermsReader(dictIn, offset, minWeight, maxWeight, type));
       }
       CodecUtil.checkFooter(index);
       success = true;
@@ -161,7 +163,11 @@ final class CompletionFieldsProducer ext
 
   @Override
   public Terms terms(String field) throws IOException {
-    return new CompletionTerms(delegateFieldsProducer.terms(field), readers.get(field));
+    Terms terms = delegateFieldsProducer.terms(field) ;
+    if (terms == null) {
+      return null;
+    }
+    return new CompletionTerms(terms, readers.get(field));
   }
 
   @Override
@@ -169,60 +175,4 @@ final class CompletionFieldsProducer ext
     return readers.size();
   }
 
-  private class CompletionsTermsReader implements Accountable {
-    private final long offset;
-    private NRTSuggester suggester;
-
-    public CompletionsTermsReader(long offset) throws IOException {
-      assert offset >= 0l && offset < dictIn.length();
-      this.offset = offset;
-    }
-
-    public synchronized NRTSuggester suggester() throws IOException {
-      if (suggester == null) {
-        try (IndexInput dictClone = dictIn.clone()) { // let multiple fields load concurrently
-          dictClone.seek(offset);
-          suggester = NRTSuggester.load(dictClone);
-        }
-      }
-      return suggester;
-    }
-
-    @Override
-    public long ramBytesUsed() {
-      return (suggester != null) ? suggester.ramBytesUsed() : 0;
-    }
-
-    @Override
-    public Collection<Accountable> getChildResources() {
-      return Collections.emptyList();
-    }
-  }
-
-  /**
-   * Thin wrapper over {@link org.apache.lucene.index.Terms} with
-   * a {@link NRTSuggester}
-   */
-  public static class CompletionTerms extends FilterLeafReader.FilterTerms {
-
-    private final CompletionsTermsReader reader;
-
-    public CompletionTerms(Terms in, CompletionsTermsReader reader) {
-      super(in);
-      this.reader = reader;
-    }
-
-    /**
-     * Returns a {@link NRTSuggester} for the field
-     * or <code>null</code> if no FST
-     * was indexed for this field
-     */
-    public NRTSuggester suggester() throws IOException {
-      if (reader == null) {
-        return null;
-      }
-      return reader.suggester();
-    }
-  }
-
 }

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java Thu May 28 08:31:45 2015
@@ -68,9 +68,12 @@ import org.apache.lucene.util.fst.FST;
  *   <li>CompletionIndex (.cmp) --&gt; Header, NumSuggestFields, Entry<sup>NumSuggestFields</sup>, Footer</li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>NumSuggestFields --&gt; {@link DataOutput#writeVInt Uint32}</li>
- *   <li>Entry --&gt; FieldNumber, CompletionDictionaryOffset</li>
+ *   <li>Entry --&gt; FieldNumber, CompletionDictionaryOffset, MinWeight, MaxWeight, Type</li>
  *   <li>FieldNumber --&gt; {@link DataOutput#writeVInt Uint32}</li>
  *   <li>CompletionDictionaryOffset --&gt; {@link DataOutput#writeVLong  Uint64}</li>
+ *   <li>MinWeight --&gt; {@link DataOutput#writeVLong  Uint64}</li>
+ *   <li>MaxWeight --&gt; {@link DataOutput#writeVLong  Uint64}</li>
+ *   <li>Type --&gt; {@link DataOutput#writeByte  Byte}</li>
  *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * <p>Notes:</p>
@@ -80,6 +83,8 @@ import org.apache.lucene.util.fst.FST;
  *   <li>NumSuggestFields is the number of suggest fields indexed</li>
  *   <li>FieldNumber is the fields number from {@link FieldInfos}. (.fnm)</li>
  *   <li>CompletionDictionaryOffset is the file offset of a field's FST in CompletionDictionary (.lkp)</li>
+ *   <li>MinWeight and MaxWeight are the global minimum and maximum weight for the field</li>
+ *   <li>Type indicates if the suggester has context or not</li>
  * </ul>
  *
  * @lucene.experimental

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java Thu May 28 08:31:45 2015
@@ -18,7 +18,7 @@ package org.apache.lucene.search.suggest
  */
 
 import java.io.IOException;
-import java.util.HashSet;
+import java.util.BitSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Set;
@@ -52,18 +52,18 @@ import static org.apache.lucene.search.s
  * The token stream uses a {@link org.apache.lucene.analysis.tokenattributes.PayloadAttribute} to store
  * a completion's payload (see {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)})
  *
+ * @lucene.experimental
  */
-final class CompletionTokenStream extends TokenStream {
+public final class CompletionTokenStream extends TokenStream {
 
   private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
   private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
   private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);
 
   private final TokenStream input;
-  private final boolean preserveSep;
-  private final boolean preservePositionIncrements;
-  private final int sepLabel;
-  private final int maxGraphExpansions;
+  final boolean preserveSep;
+  final boolean preservePositionIncrements;
+  final int maxGraphExpansions;
 
   private BytesRef payload;
   private Iterator<IntsRef> finiteStrings;
@@ -77,29 +77,20 @@ final class CompletionTokenStream extend
    * The token stream <code>input</code> is converted to an automaton
    * with the default settings of {@link org.apache.lucene.search.suggest.document.CompletionAnalyzer}
    */
-  public CompletionTokenStream(TokenStream input) {
-    this(input, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, SEP_LABEL, DEFAULT_MAX_GRAPH_EXPANSIONS);
+  CompletionTokenStream(TokenStream input) {
+    this(input, DEFAULT_PRESERVE_SEP, DEFAULT_PRESERVE_POSITION_INCREMENTS, DEFAULT_MAX_GRAPH_EXPANSIONS);
   }
 
-  CompletionTokenStream(TokenStream input, boolean preserveSep, boolean preservePositionIncrements, int sepLabel, int maxGraphExpansions) {
+  CompletionTokenStream(TokenStream input, boolean preserveSep, boolean preservePositionIncrements, int maxGraphExpansions) {
     // Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
     // the input stream entirely in toFiniteStrings(input)
     this.input = input;
     this.preserveSep = preserveSep;
     this.preservePositionIncrements = preservePositionIncrements;
-    this.sepLabel = sepLabel;
     this.maxGraphExpansions = maxGraphExpansions;
   }
 
   /**
-   * Returns a separator label that is reserved for the payload
-   * in {@link CompletionTokenStream#setPayload(org.apache.lucene.util.BytesRef)}
-   */
-  public int sepLabel() {
-    return sepLabel;
-  }
-
-  /**
    * Sets a payload available throughout successive token stream enumeration
    */
   public void setPayload(BytesRef payload) {
@@ -111,7 +102,7 @@ final class CompletionTokenStream extend
     clearAttributes();
     if (finiteStrings == null) {
       //TODO: make this return a Iterator<IntsRef> instead?
-      Automaton automaton = toAutomaton(input);
+      Automaton automaton = toAutomaton();
       Set<IntsRef> strings = Operations.getFiniteStrings(automaton, maxGraphExpansions);
 
       posInc = strings.size();
@@ -165,9 +156,17 @@ final class CompletionTokenStream extend
   }
 
   /**
-   * Converts <code>tokenStream</code> to an automaton
+   * Converts the token stream to an automaton,
+   * treating the transition labels as utf-8
+   */
+  public Automaton toAutomaton() throws IOException {
+    return toAutomaton(false);
+  }
+
+  /**
+   * Converts the tokenStream to an automaton
    */
-  public Automaton toAutomaton(TokenStream tokenStream) throws IOException {
+  public Automaton toAutomaton(boolean unicodeAware) throws IOException {
     // TODO refactor this
     // maybe we could hook up a modified automaton from TermAutomatonQuery here?
     Automaton automaton = null;
@@ -184,10 +183,11 @@ final class CompletionTokenStream extend
         tsta = new TokenStreamToAutomaton();
       }
       tsta.setPreservePositionIncrements(preservePositionIncrements);
+      tsta.setUnicodeArcs(unicodeAware);
 
-      automaton = tsta.toAutomaton(tokenStream);
+      automaton = tsta.toAutomaton(input);
     } finally {
-      IOUtils.closeWhileHandlingException(tokenStream);
+      IOUtils.closeWhileHandlingException(input);
     }
 
     // TODO: we can optimize this somewhat by determinizing
@@ -281,11 +281,12 @@ final class CompletionTokenStream extend
   }
 
   private static int[] topoSortStates(Automaton a) {
-    int[] states = new int[a.getNumStates()];
-    final Set<Integer> visited = new HashSet<>();
+    int numStates = a.getNumStates();
+    int[] states = new int[numStates];
+    final BitSet visited = new BitSet(numStates);
     final LinkedList<Integer> worklist = new LinkedList<>();
     worklist.add(0);
-    visited.add(0);
+    visited.set(0);
     int upto = 0;
     states[upto] = 0;
     upto++;
@@ -293,10 +294,10 @@ final class CompletionTokenStream extend
     while (worklist.size() > 0) {
       int s = worklist.removeFirst();
       int count = a.initTransition(s, t);
-      for (int i = 0; i < count; i++) {
+      for (int i=0;i<count;i++) {
         a.getNextTransition(t);
-        if (!visited.contains(t.dest)) {
-          visited.add(t.dest);
+        if (!visited.get(t.dest)) {
+          visited.set(t.dest);
           worklist.add(t.dest);
           states[upto++] = t.dest;
         }
@@ -305,21 +306,37 @@ final class CompletionTokenStream extend
     return states;
   }
 
-  public interface ByteTermAttribute extends TermToBytesRefAttribute {
+  /**
+   * Attribute providing access to the term builder and UTF-16 conversion
+   */
+  private interface ByteTermAttribute extends TermToBytesRefAttribute {
     // marker interface
 
     /**
-     * Return the builder from which the term is derived.
+     * Returns the builder from which the term is derived.
      */
-    public BytesRefBuilder builder();
+    BytesRefBuilder builder();
 
-    public CharSequence toUTF16();
+    /**
+     * Returns the term represented as UTF-16
+     */
+    CharSequence toUTF16();
   }
 
+  /**
+   * Custom attribute implementation for completion token stream
+   */
   public static final class ByteTermAttributeImpl extends AttributeImpl implements ByteTermAttribute, TermToBytesRefAttribute {
     private final BytesRefBuilder bytes = new BytesRefBuilder();
     private CharsRefBuilder charsRef;
 
+    /**
+     * Sole constructor
+     * no-op
+     */
+    public ByteTermAttributeImpl() {
+    }
+
     @Override
     public void fillBytesRef() {
       // does nothing - we change in place

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggester.java Thu May 28 08:31:45 2015
@@ -23,19 +23,13 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.search.CollectionTerminatedException;
-import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.suggest.analyzing.FSTUtil;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs;
@@ -48,18 +42,11 @@ import static org.apache.lucene.search.s
 
 /**
  * <p>
- * NRTSuggester returns Top N completions with corresponding documents matching a provided automaton.
- * The completions are returned in descending order of their corresponding weight.
- * Deleted documents are filtered out in near real time using the provided reader.
- * A {@link org.apache.lucene.search.DocIdSet} can be passed in at query time to filter out documents.
- * </p>
+ * NRTSuggester executes Top N search on a weighted FST specified by a {@link CompletionScorer}
  * <p>
- * See {@link #lookup(LeafReader, Automaton, int, DocIdSet, TopSuggestDocsCollector)} for more implementation
+ * See {@link #lookup(CompletionScorer, TopSuggestDocsCollector)} for more implementation
  * details.
  * <p>
- * Builder: {@link NRTSuggesterBuilder}
- * </p>
- * <p>
  * FST Format:
  * <ul>
  *   <li>Input: analyzed forms of input terms</li>
@@ -68,16 +55,17 @@ import static org.apache.lucene.search.s
  * <p>
  * NOTE:
  * <ul>
- *   <li>currently only {@link org.apache.lucene.search.DocIdSet} with random access capabilities are supported.</li>
  *   <li>having too many deletions or using a very restrictive filter can make the search inadmissible due to
- *     over-pruning of potential paths</li>
- *   <li>when a {@link org.apache.lucene.search.DocIdSet} is used, it is assumed that the filter will roughly
- *     filter out half the number of documents that match the provided automaton</li>
+ *     over-pruning of potential paths. See {@link CompletionScorer#accept(int)}</li>
+ *   <li>when matched documents are arbitrarily filtered ({@link CompletionScorer#filtered} set to <code>true</code>,
+ *     it is assumed that the filter will roughly filter out half the number of documents that match
+ *     the provided automaton</li>
  *   <li>lookup performance will degrade as more accepted completions lead to filtered out documents</li>
  * </ul>
  *
+ * @lucene.experimental
  */
-final class NRTSuggester implements Accountable {
+public final class NRTSuggester implements Accountable {
 
   /**
    * FST<Weight,Surface>:
@@ -113,7 +101,7 @@ final class NRTSuggester implements Acco
    *
    * NOTE: value should be <= Integer.MAX_VALUE
    */
-  private static final long MAX_TOP_N_QUEUE_SIZE = 1000;
+  private static final long MAX_TOP_N_QUEUE_SIZE = 5000;
 
   private NRTSuggester(FST<Pair<Long, BytesRef>> fst, int maxAnalyzedPathsPerOutput, int payloadSep, int endByte) {
     this.fst = fst;
@@ -132,102 +120,90 @@ final class NRTSuggester implements Acco
     return Collections.emptyList();
   }
 
-  private static Comparator<Pair<Long, BytesRef>> getComparator() {
-    return new Comparator<Pair<Long, BytesRef>>() {
-      @Override
-      public int compare(Pair<Long, BytesRef> o1, Pair<Long, BytesRef> o2) {
-        return Long.compare(o1.output1, o2.output1);
-      }
-    };
-  }
-
   /**
-   * Collects at most Top <code>num</code> completions, filtered by <code>filter</code> on
-   * corresponding documents, which has a prefix accepted by <code>automaton</code>
+   * Collects at most {@link TopSuggestDocsCollector#getCountToCollect()} completions that
+   * match the provided {@link CompletionScorer}.
    * <p>
-   * Supports near real time deleted document filtering using <code>reader</code>
-   * <p>
-   * {@link TopSuggestDocsCollector#collect(int, CharSequence, long)} is called
-   * for every matched completion
-   * <p>
-   * Completion collection can be early terminated by throwing {@link org.apache.lucene.search.CollectionTerminatedException}
+   * The {@link CompletionScorer#automaton} is intersected with the {@link #fst}.
+   * {@link CompletionScorer#weight} is used to compute boosts and/or extract context
+   * for each matched partial paths. A top N search is executed on {@link #fst} seeded with
+   * the matched partial paths. Upon reaching a completed path, {@link CompletionScorer#accept(int)}
+   * and {@link CompletionScorer#score(float, float)} is used on the document id, index weight
+   * and query boost to filter and score the entry, before being collected via
+   * {@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}
    */
-  public void lookup(final LeafReader reader, final Automaton automaton, final int num, final DocIdSet filter, final TopSuggestDocsCollector collector) {
-    final Bits filterDocs;
-    try {
-      if (filter != null) {
-        if (filter.iterator() == null) {
-          return;
+  public void lookup(final CompletionScorer scorer, final TopSuggestDocsCollector collector) throws IOException {
+    final double liveDocsRatio = calculateLiveDocRatio(scorer.reader.numDocs(), scorer.reader.maxDoc());
+    if (liveDocsRatio == -1) {
+      return;
+    }
+    final List<FSTUtil.Path<Pair<Long, BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(scorer.automaton, fst);
+    final int queueSize = getMaxTopNSearcherQueueSize(collector.getCountToCollect() * prefixPaths.size(),
+        scorer.reader.numDocs(), liveDocsRatio, scorer.filtered);
+    Comparator<Pair<Long, BytesRef>> comparator = getComparator();
+    Util.TopNSearcher<Pair<Long, BytesRef>> searcher = new Util.TopNSearcher<Pair<Long, BytesRef>>(fst,
+        collector.getCountToCollect(), queueSize, comparator, new ScoringPathComparator(scorer)) {
+
+      private final CharsRefBuilder spare = new CharsRefBuilder();
+
+      @Override
+      protected boolean acceptResult(Util.FSTPath<Pair<Long, BytesRef>> path) {
+        int payloadSepIndex = parseSurfaceForm(path.cost.output2, payloadSep, spare);
+        int docID = parseDocID(path.cost.output2, payloadSepIndex);
+        if (!scorer.accept(docID)) {
+          return false;
         }
-        if (filter.bits() == null) {
-          throw new IllegalArgumentException("DocIDSet does not provide random access interface");
-        } else {
-          filterDocs = filter.bits();
+        try {
+          float score = scorer.score(decode(path.cost.output1), path.boost);
+          collector.collect(docID, spare.toCharsRef(), path.context, score);
+          return true;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
         }
-      } else {
-        filterDocs = null;
       }
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+    };
 
-    int queueSize = getMaxTopNSearcherQueueSize(num, reader, filterDocs != null);
-    if (queueSize == -1) {
-      return;
+    for (FSTUtil.Path<Pair<Long, BytesRef>> path : prefixPaths) {
+      scorer.weight.setNextMatch(path.input.get());
+      searcher.addStartPaths(path.fstNode, path.output, false, path.input, scorer.weight.boost(),
+          scorer.weight.context());
     }
+    // hits are also returned by search()
+    // we do not use it, instead collect at acceptResult
+    Util.TopResults<Pair<Long, BytesRef>> search = searcher.search();
+    // search admissibility is not guaranteed
+    // see comment on getMaxTopNSearcherQueueSize
+    // assert  search.isComplete;
+  }
 
-    final Bits liveDocs = reader.getLiveDocs();
-    try {
-      final List<FSTUtil.Path<Pair<Long, BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(automaton, fst);
-      Util.TopNSearcher<Pair<Long, BytesRef>> searcher = new Util.TopNSearcher<Pair<Long, BytesRef>>(fst, num, queueSize, getComparator()) {
-
-        private final CharsRefBuilder spare = new CharsRefBuilder();
-
-        @Override
-        protected boolean acceptResult(IntsRef input, Pair<Long, BytesRef> output) {
-          int payloadSepIndex = parseSurfaceForm(output.output2, payloadSep, spare);
-          int docID = parseDocID(output.output2, payloadSepIndex);
-
-          // filter out deleted docs only if no filter is set
-          if (filterDocs == null && liveDocs != null && !liveDocs.get(docID)) {
-            return false;
-          }
-
-          // filter by filter context
-          if (filterDocs != null && !filterDocs.get(docID)) {
-            return false;
-          }
-
-          try {
-            collector.collect(docID, spare.toCharsRef(), decode(output.output1));
-            return true;
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      };
-
-      // TODO: add fuzzy support
-      for (FSTUtil.Path<Pair<Long, BytesRef>> path : prefixPaths) {
-        searcher.addStartPaths(path.fstNode, path.output, false, path.input);
-      }
+  /**
+   * Compares partial completion paths using {@link CompletionScorer#score(float, float)},
+   * breaks ties comparing path inputs
+   */
+  private static class ScoringPathComparator implements Comparator<Util.FSTPath<Pair<Long, BytesRef>>> {
+    private final CompletionScorer scorer;
 
-      try {
-        // hits are also returned by search()
-        // we do not use it, instead collect at acceptResult
-        Util.TopResults<Pair<Long, BytesRef>> search = searcher.search();
-        // search admissibility is not guaranteed
-        // see comment on getMaxTopNSearcherQueueSize
-        // assert  search.isComplete;
-      } catch (CollectionTerminatedException e) {
-        // terminate
-      }
+    public ScoringPathComparator(CompletionScorer scorer) {
+      this.scorer = scorer;
+    }
 
-    } catch (IOException bogus) {
-      throw new RuntimeException(bogus);
+    @Override
+    public int compare(Util.FSTPath<Pair<Long, BytesRef>> first, Util.FSTPath<Pair<Long, BytesRef>> second) {
+      int cmp = Float.compare(scorer.score(decode(second.cost.output1), second.boost),
+          scorer.score(decode(first.cost.output1), first.boost));
+      return (cmp != 0) ? cmp : first.input.get().compareTo(second.input.get());
     }
   }
 
+  private static Comparator<Pair<Long, BytesRef>> getComparator() {
+    return new Comparator<Pair<Long, BytesRef>>() {
+      @Override
+      public int compare(Pair<Long, BytesRef> o1, Pair<Long, BytesRef> o2) {
+        return Long.compare(o1.output1, o2.output1);
+      }
+    };
+  }
+
   /**
    * Simple heuristics to try to avoid over-pruning potential suggestions by the
    * TopNSearcher. Since suggestion entries can be rejected if they belong
@@ -241,17 +217,13 @@ final class NRTSuggester implements Acco
    * <p>
    * The maximum queue size is {@link #MAX_TOP_N_QUEUE_SIZE}
    */
-  private int getMaxTopNSearcherQueueSize(int num, LeafReader reader, boolean filterEnabled) {
-    double liveDocsRatio = calculateLiveDocRatio(reader.numDocs(), reader.maxDoc());
-    if (liveDocsRatio == -1) {
-      return -1;
-    }
-    long maxQueueSize = num * maxAnalyzedPathsPerOutput;
+  private int getMaxTopNSearcherQueueSize(int topN, int numDocs, double liveDocsRatio, boolean filterEnabled) {
+    long maxQueueSize = topN * maxAnalyzedPathsPerOutput;
     // liveDocRatio can be at most 1.0 (if no docs were deleted)
     assert liveDocsRatio <= 1.0d;
     maxQueueSize = (long) (maxQueueSize / liveDocsRatio);
     if (filterEnabled) {
-      maxQueueSize = maxQueueSize + (reader.numDocs()/2);
+      maxQueueSize = maxQueueSize + (numDocs/2);
     }
     return (int) Math.min(MAX_TOP_N_QUEUE_SIZE, maxQueueSize);
   }
@@ -276,14 +248,16 @@ final class NRTSuggester implements Acco
   }
 
   static long encode(long input) {
-    if (input < 0) {
+    if (input < 0 || input > Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("cannot encode value: " + input);
     }
-    return Long.MAX_VALUE - input;
+    return Integer.MAX_VALUE - input;
   }
 
   static long decode(long output) {
-    return (Long.MAX_VALUE - output);
+    assert output >= 0 && output <= Integer.MAX_VALUE :
+        "decoded output: " + output + " is not within 0 and Integer.MAX_VALUE";
+    return Integer.MAX_VALUE - output;
   }
 
   /**
@@ -307,7 +281,8 @@ final class NRTSuggester implements Acco
 
     static int parseDocID(final BytesRef output, int payloadSepIndex) {
       assert payloadSepIndex != -1 : "payload sep index can not be -1";
-      ByteArrayDataInput input = new ByteArrayDataInput(output.bytes, payloadSepIndex + output.offset + 1, output.length - (payloadSepIndex + output.offset));
+      ByteArrayDataInput input = new ByteArrayDataInput(output.bytes, payloadSepIndex + output.offset + 1,
+          output.length - (payloadSepIndex + output.offset));
       return input.readVInt();
     }
 

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/NRTSuggesterBuilder.java Thu May 28 08:31:45 2015
@@ -49,7 +49,7 @@ final class NRTSuggesterBuilder {
    * Marks end of the analyzed input and start of dedup
    * byte.
    */
-  private static final int END_BYTE = 0x0;
+  public static final int END_BYTE = 0x0;
 
   private final PairOutputs<Long, BytesRef> outputs;
   private final Builder<PairOutputs.Pair<Long, BytesRef>> builder;

Copied: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java (from r1682158, lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java?p2=lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java&p1=lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java&r1=1682158&r2=1682170&rev=1682170&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java Thu May 28 08:31:45 2015
@@ -60,7 +60,7 @@ public class RegexCompletionQuery extend
   /**
    * Calls {@link RegexCompletionQuery#RegexCompletionQuery(Term, int, int, Filter)}
    * enabling all optional regex syntax and <code>maxDeterminizedStates</code> of
-   * {@value Operations#DEFAULT_MAX_DETERMINIZED_STATES}
+   * {@value org.apache.lucene.util.automaton.Operations#DEFAULT_MAX_DETERMINIZED_STATES}
    */
   public RegexCompletionQuery(Term term, Filter filter) {
     this(term, RegExp.ALL, Operations.DEFAULT_MAX_DETERMINIZED_STATES, filter);

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java Thu May 28 08:31:45 2015
@@ -48,20 +48,14 @@ import org.apache.lucene.util.BytesRef;
  * document.add(new SuggestField(name, "suggestion", 4));
  * </pre>
  * To perform document suggestions based on the this field, use
- * {@link SuggestIndexSearcher#suggest(String, CharSequence, int, org.apache.lucene.search.Filter)}
- * <p>
- * Example query usage:
- * <pre class="prettyprint">
- * SuggestIndexSearcher indexSearcher = ..
- * indexSearcher.suggest(name, "su", 2)
- * </pre>
+ * {@link SuggestIndexSearcher#suggest(CompletionQuery, int)}
  *
  * @lucene.experimental
  */
 public class SuggestField extends Field {
 
-  private static final FieldType FIELD_TYPE = new FieldType();
-
+  /** Default field type for suggest field */
+  public static final FieldType FIELD_TYPE = new FieldType();
   static {
     FIELD_TYPE.setTokenized(true);
     FIELD_TYPE.setStored(false);
@@ -71,53 +65,86 @@ public class SuggestField extends Field
     FIELD_TYPE.freeze();
   }
 
+  static final byte TYPE = 0;
+
   private final BytesRef surfaceForm;
-  private final long weight;
+  private final int weight;
 
   /**
    * Creates a {@link SuggestField}
    *
-   * @param name   of the field
-   * @param value  to get suggestions on
-   * @param weight weight of the suggestion
+   * @param name   field name
+   * @param value  field value to get suggestions on
+   * @param weight field weight
+   *
+   * @throws IllegalArgumentException if either the name or value is null,
+   * if value is an empty string, if the weight is negative, if value contains
+   * any reserved characters
    */
-  public SuggestField(String name, String value, long weight) {
+  public SuggestField(String name, String value, int weight) {
     super(name, value, FIELD_TYPE);
-    if (weight < 0l) {
+    if (weight < 0) {
       throw new IllegalArgumentException("weight must be >= 0");
     }
+    if (value.length() == 0) {
+      throw new IllegalArgumentException("value must have a length > 0");
+    }
+    for (int i = 0; i < value.length(); i++) {
+      if (isReserved(value.charAt(i))) {
+        throw new IllegalArgumentException("Illegal input [" + value + "] UTF-16 codepoint [0x"
+            + Integer.toHexString((int) value.charAt(i))+ "] at position " + i + " is a reserved character");
+      }
+    }
     this.surfaceForm = new BytesRef(value);
     this.weight = weight;
   }
 
   @Override
   public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) throws IOException {
-    TokenStream stream = super.tokenStream(analyzer, reuse);
-    CompletionTokenStream completionStream;
+    CompletionTokenStream completionStream = wrapTokenStream(super.tokenStream(analyzer, reuse));
+    completionStream.setPayload(buildSuggestPayload());
+    return completionStream;
+  }
+
+  /**
+   * Wraps a <code>stream</code> with a CompletionTokenStream.
+   *
+   * Subclasses can override this method to change the indexing pipeline.
+   */
+  protected CompletionTokenStream wrapTokenStream(TokenStream stream) {
     if (stream instanceof CompletionTokenStream) {
-      completionStream = (CompletionTokenStream) stream;
+      return (CompletionTokenStream) stream;
     } else {
-      completionStream = new CompletionTokenStream(stream);
+      return new CompletionTokenStream(stream);
     }
-    BytesRef suggestPayload = buildSuggestPayload(surfaceForm, weight, (char) completionStream.sepLabel());
-    completionStream.setPayload(suggestPayload);
-    return completionStream;
   }
 
-  private BytesRef buildSuggestPayload(BytesRef surfaceForm, long weight, char sepLabel) throws IOException {
-    for (int i = 0; i < surfaceForm.length; i++) {
-      if (surfaceForm.bytes[i] == sepLabel) {
-        assert sepLabel == '\u001f';
-        throw new IllegalArgumentException(
-            "surface form cannot contain unit separator character U+001F; this character is reserved");
-      }
-    }
+  /**
+   * Returns a byte to denote the type of the field
+   */
+  protected byte type() {
+    return TYPE;
+  }
+
+  private BytesRef buildSuggestPayload() throws IOException {
     ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
     try (OutputStreamDataOutput output = new OutputStreamDataOutput(byteArrayOutputStream)) {
       output.writeVInt(surfaceForm.length);
       output.writeBytes(surfaceForm.bytes, surfaceForm.offset, surfaceForm.length);
-      output.writeVLong(weight + 1);
+      output.writeVInt(weight + 1);
+      output.writeByte(type());
     }
     return new BytesRef(byteArrayOutputStream.toByteArray());
   }
+
+  private boolean isReserved(char c) {
+    switch (c) {
+      case CompletionAnalyzer.SEP_LABEL:
+      case CompletionAnalyzer.HOLE_CHARACTER:
+      case NRTSuggesterBuilder.END_BYTE:
+        return true;
+      default:
+        return false;
+    }
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestIndexSearcher.java Thu May 28 08:31:45 2015
@@ -19,132 +19,66 @@ package org.apache.lucene.search.suggest
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.CollectionTerminatedException;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.automaton.Automaton;
-
-import static org.apache.lucene.search.suggest.document.CompletionFieldsProducer.CompletionTerms;
+import org.apache.lucene.search.Weight;
 
 /**
- * Adds document suggest capabilities to IndexSearcher
+ * Adds document suggest capabilities to IndexSearcher.
+ * Any {@link CompletionQuery} can be used to suggest documents.
+ *
+ * Use {@link PrefixCompletionQuery} for analyzed prefix queries,
+ * {@link RegexCompletionQuery} for regular expression prefix queries,
+ * {@link FuzzyCompletionQuery} for analyzed prefix with typo tolerance
+ * and {@link ContextQuery} to boost and/or filter suggestions by contexts
  *
  * @lucene.experimental
  */
 public class SuggestIndexSearcher extends IndexSearcher {
 
-  private final Analyzer queryAnalyzer;
-
   /**
    * Creates a searcher with document suggest capabilities
    * for <code>reader</code>.
-   * <p>
-   * Suggestion <code>key</code> is analyzed with <code>queryAnalyzer</code>
    */
-  public SuggestIndexSearcher(IndexReader reader, Analyzer queryAnalyzer) {
+  public SuggestIndexSearcher(IndexReader reader) {
     super(reader);
-    this.queryAnalyzer = queryAnalyzer;
-  }
-
-  /**
-   * Calls {@link #suggest(String, CharSequence, int, Filter)}
-   * with no document filter
-   */
-  public TopSuggestDocs suggest(String field, CharSequence key, int num) throws IOException {
-    return suggest(field, key, num, (Filter) null);
-  }
-
-  /**
-   * Calls {@link #suggest(String, CharSequence, int, Filter, TopSuggestDocsCollector)}
-   * with no document filter
-   */
-  public void suggest(String field, CharSequence key, int num, TopSuggestDocsCollector collector) throws IOException {
-    suggest(field, key, num, null, collector);
   }
 
   /**
-   * Suggests at most <code>num</code> documents filtered by <code>filter</code>
-   * that completes to <code>key</code> for a suggest <code>field</code>
-   * <p>
-   * Returns at most Top <code>num</code> document ids with corresponding completion and weight pair
-   *
-   * @throws java.lang.IllegalArgumentException if <code>filter</code> does not provide a random access
-   *                                            interface or if <code>field</code> is not a {@link SuggestField}
+   * Returns top <code>n</code> completion hits for
+   * <code>query</code>
    */
-  public TopSuggestDocs suggest(String field, CharSequence key, int num, Filter filter) throws IOException {
-    TopSuggestDocsCollector collector = new TopSuggestDocsCollector(num);
-    suggest(field, key, num, filter, collector);
+  public TopSuggestDocs suggest(CompletionQuery query, int n) throws IOException {
+    TopSuggestDocsCollector collector = new TopSuggestDocsCollector(n);
+    suggest(query, collector);
     return collector.get();
   }
 
   /**
-   * Suggests at most <code>num</code> documents filtered by <code>filter</code>
-   * that completes to <code>key</code> for a suggest <code>field</code>
-   * <p>
-   * Collect completions with {@link TopSuggestDocsCollector}
-   * The completions are collected in order of the suggest <code>field</code> weight.
-   * There can be more than one collection of the same document, if the <code>key</code>
-   * matches multiple <code>field</code> values of the same document
+   * Lower-level suggest API.
+   * Collects completion hits through <code>collector</code> for <code>query</code>.
    *
-   * @throws java.lang.IllegalArgumentException if <code>filter</code> does not provide a random access
-   *                                            interface or if <code>field</code> is not a {@link SuggestField}
+   * <p>{@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}
+   * is called for every matching completion hit.
    */
-  public void suggest(String field, CharSequence key, int num, Filter filter, TopSuggestDocsCollector collector) throws IOException {
-    // verify input
-    if (field == null) {
-      throw new IllegalArgumentException("'field' can not be null");
-    }
-    if (num <= 0) {
-      throw new IllegalArgumentException("'num' should be > 0");
-    }
-    if (collector == null) {
-      throw new IllegalArgumentException("'collector' can not be null");
-    }
-
-    // build query automaton
-    CompletionAnalyzer analyzer;
-    if (queryAnalyzer instanceof CompletionAnalyzer) {
-      analyzer = (CompletionAnalyzer) queryAnalyzer;
-    } else {
-      analyzer = new CompletionAnalyzer(queryAnalyzer);
-    }
-    final Automaton automaton = analyzer.toAutomaton(field, key);
-
-    // collect results
+  public void suggest(CompletionQuery query, TopSuggestDocsCollector collector) throws IOException {
+    // TODO use IndexSearcher.rewrite instead
+    // have to implement equals() and hashCode() in CompletionQuerys and co
+    query = (CompletionQuery) query.rewrite(getIndexReader());
+    Weight weight = query.createWeight(this, collector.needsScores());
     for (LeafReaderContext context : getIndexReader().leaves()) {
-      TopSuggestDocsCollector leafCollector = (TopSuggestDocsCollector) collector.getLeafCollector(context);
-      LeafReader reader = context.reader();
-      Terms terms = reader.terms(field);
-      if (terms == null) {
-        continue;
-      }
-      NRTSuggester suggester;
-      if (terms instanceof CompletionTerms) {
-        CompletionTerms completionTerms = (CompletionTerms) terms;
-        suggester = completionTerms.suggester();
-      } else {
-        throw new IllegalArgumentException(field + " is not a SuggestField");
-      }
-      if (suggester == null) {
-        // a segment can have a null suggester
-        // i.e. no FST was built
-        continue;
-      }
-
-      DocIdSet docIdSet = null;
-      if (filter != null) {
-        docIdSet = filter.getDocIdSet(context, reader.getLiveDocs());
-        if (docIdSet == null) {
-          // filter matches no docs in current leave
-          continue;
+      BulkScorer scorer = weight.bulkScorer(context, context.reader().getLiveDocs());
+      if (scorer != null) {
+        try {
+          scorer.score(collector.getLeafCollector(context));
+        } catch (CollectionTerminatedException e) {
+          // collection was terminated prematurely
+          // continue with the following leaf
         }
       }
-      suggester.lookup(reader, automaton, num, docIdSet, leafCollector);
     }
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocs.java Thu May 28 08:31:45 2015
@@ -43,7 +43,12 @@ public class TopSuggestDocs extends TopD
     /**
      * Matched completion key
      */
-    public CharSequence key;
+    public final CharSequence key;
+
+    /**
+     * Context for the completion
+     */
+    public final CharSequence context;
 
     /**
      * Creates a SuggestScoreDoc instance
@@ -52,11 +57,10 @@ public class TopSuggestDocs extends TopD
      * @param key   matched completion
      * @param score weight of the matched completion
      */
-    public SuggestScoreDoc(int doc, CharSequence key, long score) {
-      // loss of precision but not magnitude
-      // implicit conversion from long -> float
+    public SuggestScoreDoc(int doc, CharSequence key, CharSequence context, float score) {
       super(doc, score);
       this.key = key;
+      this.context = context;
     }
 
     @Override

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java?rev=1682170&r1=1682169&r2=1682170&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/TopSuggestDocsCollector.java Thu May 28 08:31:45 2015
@@ -30,20 +30,23 @@ import static org.apache.lucene.search.s
  * score, along with document id
  * <p>
  * Non scoring collector that collect completions in order of their
- * pre-defined weight.
+ * pre-computed scores.
  * <p>
  * NOTE: One document can be collected multiple times if a document
  * is matched for multiple unique completions for a given query
  * <p>
- * Subclasses should only override {@link TopSuggestDocsCollector#collect(int, CharSequence, long)},
- * {@link #setScorer(org.apache.lucene.search.Scorer)} is not
- * used
+ * Subclasses should only override
+ * {@link TopSuggestDocsCollector#collect(int, CharSequence, CharSequence, float)}.
+ * <p>
+ * NOTE: {@link #setScorer(org.apache.lucene.search.Scorer)} and
+ * {@link #collect(int)} is not used
  *
  * @lucene.experimental
  */
 public class TopSuggestDocsCollector extends SimpleCollector {
 
   private final SuggestScoreDocPriorityQueue priorityQueue;
+  private final int num;
 
   /**
    * Document base offset for the current Leaf
@@ -60,9 +63,17 @@ public class TopSuggestDocsCollector ext
     if (num <= 0) {
       throw new IllegalArgumentException("'num' must be > 0");
     }
+    this.num = num;
     this.priorityQueue = new SuggestScoreDocPriorityQueue(num);
   }
 
+  /**
+   * Returns the number of results to be collected
+   */
+  public int getCountToCollect() {
+    return num;
+  }
+
   @Override
   protected void doSetNextReader(LeafReaderContext context) throws IOException {
     docBase = context.docBase;
@@ -76,8 +87,8 @@ public class TopSuggestDocsCollector ext
    * NOTE: collection at the leaf level is guaranteed to be in
    * descending order of score
    */
-  public void collect(int docID, CharSequence key, long score) throws IOException {
-    SuggestScoreDoc current = new SuggestScoreDoc(docBase + docID, key, score);
+  public void collect(int docID, CharSequence key, CharSequence context, float score) throws IOException {
+    SuggestScoreDoc current = new SuggestScoreDoc(docBase + docID, key, context, score);
     if (current == priorityQueue.insertWithOverflow(current)) {
       // if the current SuggestScoreDoc has overflown from pq,
       // we can assume all of the successive collections from
@@ -104,7 +115,7 @@ public class TopSuggestDocsCollector ext
    */
   @Override
   public void collect(int doc) throws IOException {
-    // {@link #collect(int, CharSequence, long)} is used
+    // {@link #collect(int, CharSequence, CharSequence, long)} is used
     // instead
   }
 
@@ -113,6 +124,6 @@ public class TopSuggestDocsCollector ext
    */
   @Override
   public boolean needsScores() {
-    return false;
+    return true;
   }
 }

Copied: lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java (from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java?p2=lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java&p1=lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java&r1=1682158&r2=1682170&rev=1682170&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java Thu May 28 08:31:45 2015
@@ -73,12 +73,13 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "suggestion1", 8));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 7));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 6));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 8));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 7));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 6));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 5));
+
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 5));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -114,7 +115,8 @@ public class TestContextQuery extends Lu
     document.add(new SuggestField("suggest_field", "abd", 4));
     document.add(new SuggestField("suggest_field", "The Foo Fighters", 2));
     iw.addDocument(document);
-    document.clear();
+
+    document = new Document();
     document.add(new SuggestField("suggest_field", "abcdd", 5));
     iw.addDocument(document);
 
@@ -140,12 +142,12 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 3));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 2));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 4));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -173,8 +175,8 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("typetype"), "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type"), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("typetype"), "suggestion1", 4));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type"), "suggestion2", 3));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -204,8 +206,9 @@ public class TestContextQuery extends Lu
 
     document.add(new ContextSuggestField("suggest_field", null, "suggestion_no_ctx", 4));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion", 1));
+
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -231,11 +234,11 @@ public class TestContextQuery extends Lu
     Document document = new Document();
 
     document.add(new ContextSuggestField("suggest_field", null, "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.emptyList(), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>emptyList(), "suggestion2", 3));
     document.add(new ContextSuggestField("suggest_field", null, "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -265,10 +268,10 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Arrays.asList("type1", "type2", "type3"), "suggestion", 4));
+    document.add(new ContextSuggestField("suggest_field", Arrays.<CharSequence>asList("type1", "type2", "type3"), "suggestion", 4));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -300,12 +303,12 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 3));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 2));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 4));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -336,12 +339,12 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 3));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 2));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 4));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -369,12 +372,12 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 3));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 2));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 4));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -401,12 +404,12 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Arrays.asList("type1", "type3"), "suggestion1", 8));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 7));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 6));
+    document.add(new ContextSuggestField("suggest_field", Arrays.<CharSequence>asList("type1", "type3"), "suggestion1", 8));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 7));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 6));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 5));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 5));
     iw.addDocument(document);
 
     if (rarely()) {
@@ -438,12 +441,12 @@ public class TestContextQuery extends Lu
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
     Document document = new Document();
 
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "suggestion2", 3));
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "suggestion3", 2));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 4));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
-    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document = new Document();
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {

Copied: lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java (from r1682158, lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java?p2=lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java&p1=lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java&r1=1682158&r2=1682170&rev=1682170&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java Thu May 28 08:31:45 2015
@@ -53,7 +53,7 @@ public class TestContextSuggestField ext
   @Test
   public void testEmptySuggestion() throws Exception {
     try {
-      new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "", 1);
+      new ContextSuggestField("suggest_field", Collections.<CharSequence>singletonList("type1"), "", 1);
       fail("no exception thrown when indexing zero length suggestion");
     } catch (IllegalArgumentException expected) {
       assertTrue(expected.getMessage().contains("value"));
@@ -66,14 +66,14 @@ public class TestContextSuggestField ext
     charsRefBuilder.append("sugg");
     charsRefBuilder.setCharAt(2, (char) ContextSuggestField.CONTEXT_SEPARATOR);
     try {
-      new ContextSuggestField("name", Collections.singletonList(charsRefBuilder.toString()), "sugg", 1);
+      new ContextSuggestField("name", Collections.<CharSequence>singletonList(charsRefBuilder.toString()), "sugg", 1);
       fail("no exception thrown for context value containing CONTEXT_SEPARATOR:" + ContextSuggestField.CONTEXT_SEPARATOR);
     } catch (IllegalArgumentException e) {
       assertTrue(e.getMessage().contains("[0x1d]"));
     }
 
     try {
-      new ContextSuggestField("name", Collections.singletonList("sugg"), charsRefBuilder.toString(), 1);
+      new ContextSuggestField("name", Collections.<CharSequence>singletonList("sugg"), charsRefBuilder.toString(), 1);
       fail("no exception thrown for value containing CONTEXT_SEPARATOR:" + ContextSuggestField.CONTEXT_SEPARATOR);
     } catch (IllegalArgumentException e) {
       assertTrue(e.getMessage().contains("[0x1d]"));
@@ -85,7 +85,7 @@ public class TestContextSuggestField ext
     Analyzer analyzer = new MockAnalyzer(random());
     Document document = new Document();
     document.add(new SuggestField("suggest_field", "suggestion1", 4));
-    document.add(new ContextSuggestField("suggest_field", Collections.emptyList(), "suggestion2", 3));
+    document.add(new ContextSuggestField("suggest_field", Collections.<CharSequence>emptyList(), "suggestion2", 3));
 
     try (RandomIndexWriter iw = new RandomIndexWriter(random(), dir,
         iwcWithSuggestField(analyzer, "suggest_field"))) {
@@ -107,13 +107,13 @@ public class TestContextSuggestField ext
     document.add(new SuggestField("suggest_field", "suggestion1", 4));
     document.add(new SuggestField("suggest_field", "suggestion2", 3));
     document.add(new SuggestField("suggest_field", "suggestion3", 2));
-    document.add(new ContextSuggestField("context_suggest_field", Collections.singletonList("type1"), "suggestion1", 4));
-    document.add(new ContextSuggestField("context_suggest_field", Collections.singletonList("type2"), "suggestion2", 3));
-    document.add(new ContextSuggestField("context_suggest_field", Collections.singletonList("type3"), "suggestion3", 2));
+    document.add(new ContextSuggestField("context_suggest_field", Collections.<CharSequence>singletonList("type1"), "suggestion1", 4));
+    document.add(new ContextSuggestField("context_suggest_field", Collections.<CharSequence>singletonList("type2"), "suggestion2", 3));
+    document.add(new ContextSuggestField("context_suggest_field", Collections.<CharSequence>singletonList("type3"), "suggestion3", 2));
     iw.addDocument(document);
-    document.clear();
+    document = new Document();
     document.add(new SuggestField("suggest_field", "suggestion4", 1));
-    document.add(new ContextSuggestField("context_suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    document.add(new ContextSuggestField("context_suggest_field", Collections.<CharSequence>singletonList("type4"), "suggestion4", 1));
     iw.addDocument(document);
 
     if (rarely()) {