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

svn commit: r1682158 [1/3] - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/util/fst/ suggest/src/java/org/apache/lucene/search/suggest/document/ suggest/src/test/org/apache/lucene/search/suggest/document/

Author: mikemccand
Date: Thu May 28 07:53:09 2015
New Revision: 1682158

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

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

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu May 28 07:53:09 2015
@@ -41,6 +41,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/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Thu May 28 07:53:09 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionAnalyzer.java Thu May 28 07:53:09 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsConsumer.java Thu May 28 07:53:09 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionFieldsProducer.java Thu May 28 07:53:09 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/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionPostingsFormat.java Thu May 28 07:53:09 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

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,170 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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 org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
+
+import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.HOLE_CHARACTER;
+import static org.apache.lucene.search.suggest.document.CompletionAnalyzer.SEP_LABEL;
+
+/**
+ * Abstract {@link Query} that match documents containing terms with a specified prefix
+ * filtered by {@link Filter}. This should be used to query against any {@link SuggestField}s
+ * or {@link ContextSuggestField}s of documents.
+ * <p>
+ * Use {@link SuggestIndexSearcher#suggest(CompletionQuery, int)} to execute any query
+ * that provides a concrete implementation of this query. Example below shows using this query
+ * to retrieve the top 5 documents.
+ *
+ * <pre class="prettyprint">
+ *  SuggestIndexSearcher searcher = new SuggestIndexSearcher(reader);
+ *  TopSuggestDocs suggestDocs = searcher.suggest(query, 5);
+ * </pre>
+ * This query rewrites to an appropriate {@link CompletionQuery} depending on the
+ * type ({@link SuggestField} or {@link ContextSuggestField}) of the field the query is run against.
+ *
+ * @lucene.experimental
+ */
+public abstract class CompletionQuery extends Query {
+
+  /**
+   * Term to query against
+   */
+  private final Term term;
+
+  /**
+   * Filter for document scoping
+   */
+  private final Filter filter;
+
+  /**
+   * Creates a base Completion query against a <code>term</code>
+   * with a <code>filter</code> to scope the documents
+   */
+  protected CompletionQuery(Term term, Filter filter) {
+    validate(term.text());
+    this.term = term;
+    this.filter = filter;
+  }
+
+  /**
+   * Returns the filter for the query, used to
+   * suggest completions on a subset of indexed documents
+   */
+  public Filter getFilter() {
+    return filter;
+  }
+
+  /**
+   * Returns the field name this query should
+   * be run against
+   */
+  public String getField() {
+    return term.field();
+  }
+
+  /**
+   * Returns the term to be queried against
+   */
+  public Term getTerm() {
+    return term;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    byte type = 0;
+    boolean first = true;
+    Terms terms;
+    for (LeafReaderContext context : reader.leaves()) {
+      LeafReader leafReader = context.reader();
+      try {
+        if ((terms = leafReader.terms(getField())) == null) {
+          continue;
+        }
+      } catch (IOException e) {
+        continue;
+      }
+      if (terms instanceof CompletionTerms) {
+        CompletionTerms completionTerms = (CompletionTerms) terms;
+        byte t = completionTerms.getType();
+        if (first) {
+          type = t;
+          first = false;
+        } else if (type != t) {
+          throw new IllegalStateException(getField() + " has values of multiple types");
+        }
+      }
+    }
+
+    if (first == false) {
+      if (this instanceof ContextQuery) {
+        if (type == SuggestField.TYPE) {
+          throw new IllegalStateException(this.getClass().getSimpleName()
+              + " can not be executed against a non context-enabled SuggestField: "
+              + getField());
+        }
+      } else {
+        if (type == ContextSuggestField.TYPE) {
+          return new ContextQuery(this);
+        }
+      }
+    }
+    return this;
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder buffer = new StringBuilder();
+    if (!term.field().equals(field)) {
+      buffer.append(term.field());
+      buffer.append(":");
+    }
+    buffer.append(term.text());
+    buffer.append('*');
+    if (filter != null) {
+      buffer.append(",");
+      buffer.append("filter");
+      buffer.append(":");
+      buffer.append(filter.toString(field));
+    }
+    return buffer.toString();
+  }
+
+  private void validate(String termText) {
+    for (int i = 0; i < termText.length(); i++) {
+      switch (termText.charAt(i)) {
+        case HOLE_CHARACTER:
+          throw new IllegalArgumentException(
+              "Term text cannot contain HOLE character U+001E; this character is reserved");
+        case SEP_LABEL:
+          throw new IllegalArgumentException(
+              "Term text cannot contain unit separator character U+001F; this character is reserved");
+        default:
+          break;
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionScorer.java Thu May 28 07:53:09 2015
@@ -0,0 +1,103 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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 org.apache.lucene.index.LeafReader;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.automaton.Automaton;
+
+/**
+ * Expert: Responsible for executing the query against an
+ * appropriate suggester and collecting the results
+ * via a collector.
+ *
+ * {@link #score(LeafCollector, int, int)} is called
+ * for each leaf reader.
+ *
+ * {@link #accept(int)} and {@link #score(float, float)}
+ * is called for every matched completion (i.e. document)
+ *
+ * @lucene.experimental
+ */
+public class CompletionScorer extends BulkScorer {
+  private final NRTSuggester suggester;
+  private final Bits acceptDocs;
+
+  // values accessed by suggester
+  /** weight that created this scorer */
+  protected final CompletionWeight weight;
+  final LeafReader reader;
+  final boolean filtered;
+  final Automaton automaton;
+
+  /**
+   * Creates a scorer for a field-specific <code>suggester</code> scoped by <code>acceptDocs</code>
+   */
+  protected CompletionScorer(final CompletionWeight weight, final NRTSuggester suggester,
+                             final LeafReader reader, final Bits acceptDocs,
+                             final boolean filtered, final Automaton automaton) throws IOException {
+    this.weight = weight;
+    this.suggester = suggester;
+    this.reader = reader;
+    this.automaton = automaton;
+    this.filtered = filtered;
+    this.acceptDocs = acceptDocs;
+  }
+
+  @Override
+  public int score(LeafCollector collector, int min, int max) throws IOException {
+    if (!(collector instanceof TopSuggestDocsCollector)) {
+      throw new IllegalArgumentException("collector is not of type TopSuggestDocsCollector");
+    }
+    suggester.lookup(this, ((TopSuggestDocsCollector) collector));
+    return max;
+  }
+
+  @Override
+  public long cost() {
+    return 0;
+  }
+
+  /**
+   * Returns true if a document with <code>docID</code> is accepted,
+   * false if the docID maps to a deleted
+   * document or has been filtered out
+   */
+  public final boolean accept(int docID) {
+    return acceptDocs == null || acceptDocs.get(docID);
+  }
+
+  /**
+   * Returns the score for a matched completion
+   * based on the query time boost and the
+   * index time weight.
+   */
+  public float score(float weight, float boost) {
+    if (boost == 0f) {
+      return weight;
+    }
+    if (weight == 0f) {
+      return boost;
+    }
+    return weight * boost;
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTerms.java Thu May 28 07:53:09 2015
@@ -0,0 +1,74 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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 org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.Terms;
+
+/**
+ * Wrapped {@link org.apache.lucene.index.Terms}
+ * used by {@link SuggestField} and {@link ContextSuggestField}
+ * to access corresponding suggester and their attributes
+ *
+ * @lucene.experimental
+ */
+public final class CompletionTerms extends FilterLeafReader.FilterTerms {
+
+  private final CompletionsTermsReader reader;
+
+  /**
+   * Creates a completionTerms based on {@link CompletionsTermsReader}
+   */
+  CompletionTerms(Terms in, CompletionsTermsReader reader) {
+    super(in);
+    this.reader = reader;
+  }
+
+  /**
+   * Returns the type of FST, either {@link SuggestField#TYPE} or
+   * {@link ContextSuggestField#TYPE}
+   */
+  public byte getType() {
+    return (reader != null) ? reader.type : SuggestField.TYPE;
+  }
+
+  /**
+   * Returns the minimum weight of all entries in the weighted FST
+   */
+  public long getMinWeight() {
+    return (reader != null) ? reader.minWeight : 0;
+  }
+
+  /**
+   * Returns the maximum weight of all entries in the weighted FST
+   */
+  public long getMaxWeight() {
+    return (reader != null) ? reader.maxWeight : 0;
+  }
+
+  /**
+   * Returns a {@link NRTSuggester} for the field
+   * or <code>null</code> if no FST
+   * was indexed for this field
+   */
+  public NRTSuggester suggester() throws IOException {
+    return (reader != null) ? reader.suggester() : null;
+  }
+}

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java?rev=1682158&r1=1682157&r2=1682158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionTokenStream.java Thu May 28 07:53:09 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

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java Thu May 28 07:53:09 2015
@@ -0,0 +1,209 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.automaton.Automaton;
+
+/**
+ * Expert: the Weight for CompletionQuery, used to
+ * score and explain these queries.
+ *
+ * Subclasses can override {@link #setNextMatch(IntsRef)},
+ * {@link #boost()} and {@link #context()}
+ * to calculate the boost and extract the context of
+ * a matched path prefix.
+ *
+ * @lucene.experimental
+ */
+public class CompletionWeight extends Weight {
+  private final CompletionQuery completionQuery;
+  private final Automaton automaton;
+  private final long maxWeight;
+  private final long minWeight;
+
+  /**
+   * Creates a weight for <code>query</code> with an <code>automaton</code>,
+   * using the <code>reader</code> for index stats
+   */
+  public CompletionWeight(final IndexReader reader, final CompletionQuery query,
+                          final Automaton automaton) throws IOException {
+    super(query);
+    this.completionQuery = query;
+    this.automaton = automaton;
+    this.minWeight = minWeight(query.getField(), reader);
+    this.maxWeight = maxWeight(query.getField(), reader);
+  }
+
+  /**
+   * Returns the automaton specified
+   * by the {@link CompletionQuery}
+   *
+   * @return query automaton
+   */
+  public Automaton getAutomaton() {
+    return automaton;
+  }
+
+  @Override
+  public BulkScorer bulkScorer(final LeafReaderContext context, Bits acceptDocs) throws IOException {
+    final LeafReader reader = context.reader();
+    final Terms terms;
+    final NRTSuggester suggester;
+    if ((terms = reader.terms(completionQuery.getField())) == null) {
+      return null;
+    }
+    if (terms instanceof CompletionTerms) {
+      CompletionTerms completionTerms = (CompletionTerms) terms;
+      if ((suggester = completionTerms.suggester()) == null) {
+        // a segment can have a null suggester
+        // i.e. no FST was built
+        return null;
+      }
+    } else {
+      throw new IllegalArgumentException(completionQuery.getField() + " is not a SuggestField");
+    }
+
+    DocIdSet docIdSet = null;
+    Filter filter = completionQuery.getFilter();
+    if (filter != null) {
+      docIdSet = filter.getDocIdSet(context, acceptDocs);
+      if (docIdSet == null || docIdSet.iterator() == null) {
+        // filter matches no docs in current leave
+        return null;
+      } else if (docIdSet.bits() == null) {
+        throw new IllegalArgumentException("DocIDSet does not provide random access interface");
+      }
+    }
+    Bits acceptDocBits = (docIdSet != null) ? docIdSet.bits() : acceptDocs;
+    return new CompletionScorer(this, suggester, reader, acceptDocBits, filter != null, automaton);
+  }
+
+  /**
+   * Set for every partial path in the index that matched the query
+   * automaton.
+   *
+   * Subclasses should override {@link #boost()} and {@link #context()}
+   * to return an appropriate value with respect to the current pathPrefix.
+   *
+   * @param pathPrefix the prefix of a matched path
+   */
+  protected void setNextMatch(IntsRef pathPrefix) {
+  }
+
+  /**
+   * Returns the boost of the partial path set by {@link #setNextMatch(IntsRef)}
+   *
+   * @return suggestion query-time boost
+   */
+  protected float boost() {
+    return 0;
+  }
+
+  /**
+   * Returns the context of the partial path set by {@link #setNextMatch(IntsRef)}
+   *
+   * @return suggestion context
+   */
+  protected CharSequence context() {
+    return null;
+  }
+
+  private static long minWeight(String field, IndexReader reader) {
+    long minWeight = Long.MAX_VALUE;
+    Terms terms;
+    for (LeafReaderContext context : reader.leaves()) {
+      LeafReader leafReader = context.reader();
+      try {
+        if ((terms = leafReader.terms(field)) == null) {
+          continue;
+        }
+      } catch (IOException e) {
+        continue;
+      }
+      if (terms instanceof CompletionTerms) {
+        CompletionTerms completionTerms = (CompletionTerms) terms;
+        minWeight = Math.min(completionTerms.getMinWeight(), minWeight);
+      }
+    }
+    if (minWeight == Long.MAX_VALUE) {
+      minWeight = 0;
+    }
+    return minWeight;
+  }
+
+  private static long maxWeight(String field, IndexReader reader) {
+    long maxWeight = 0;
+    Terms terms;
+    for (LeafReaderContext context : reader.leaves()) {
+      LeafReader leafReader = context.reader();
+      try {
+        if ((terms = leafReader.terms(field)) == null) {
+          continue;
+        }
+      } catch (IOException e) {
+        continue;
+      }
+      if (terms instanceof CompletionTerms) {
+        CompletionTerms completionTerms = (CompletionTerms) terms;
+        maxWeight = Math.max(completionTerms.getMaxWeight(), maxWeight);
+      }
+    }
+    return maxWeight;
+  }
+
+  @Override
+  public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void extractTerms(Set<Term> terms) {
+    // no-op
+  }
+
+  @Override
+  public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+    //TODO
+    return null;
+  }
+
+  @Override
+  public float getValueForNormalization() throws IOException {
+    return 0;
+  }
+
+  @Override
+  public void normalize(float norm, float topLevelBoost) {
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionsTermsReader.java Thu May 28 07:53:09 2015
@@ -0,0 +1,82 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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.Collection;
+import java.util.Collections;
+
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+
+/**
+ * Holder for suggester and field-level info
+ * for a suggest field
+ *
+ * @lucene.experimental
+ */
+public final class CompletionsTermsReader implements Accountable {
+  /** Minimum entry weight for the suggester */
+  public final long minWeight;
+  /** Maximum entry weight for the suggester */
+  public final long maxWeight;
+  /** type of suggester (context-enabled or not) */
+  public final byte type;
+  private final IndexInput dictIn;
+  private final long offset;
+
+  private NRTSuggester suggester;
+
+  /**
+   * Creates a CompletionTermsReader to load a field-specific suggester
+   * from the index <code>dictIn</code> with <code>offset</code>
+   */
+  CompletionsTermsReader(IndexInput dictIn, long offset, long minWeight, long maxWeight, byte type) throws IOException {
+    assert minWeight <= maxWeight;
+    assert offset >= 0l && offset < dictIn.length();
+    this.dictIn = dictIn;
+    this.offset = offset;
+    this.minWeight = minWeight;
+    this.maxWeight = maxWeight;
+    this.type = type;
+  }
+
+  /**
+   * Returns the suggester for a field, if not loaded already, loads
+   * the appropriate suggester from CompletionDictionary
+   */
+  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();
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,307 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeSet;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.RegExp;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * A {@link CompletionQuery} that match documents specified by
+ * a wrapped {@link CompletionQuery} supporting boosting and/or filtering
+ * by specified contexts.
+ * <p>
+ * Use this query against {@link ContextSuggestField}
+ * <p>
+ * Example of using a {@link CompletionQuery} with boosted
+ * contexts:
+ * <pre class="prettyprint">
+ *  CompletionQuery completionQuery = ...;
+ *  ContextQuery query = new ContextQuery(completionQuery);
+ *  query.addContext("context1", 2);
+ *  query.addContext("context2", 1);
+ * </pre>
+ * <p>
+ * NOTE:
+ * <ul>
+ *   <li>
+ *    This query can be constructed with
+ *    {@link PrefixCompletionQuery}, {@link RegexCompletionQuery}
+ *    or {@link FuzzyCompletionQuery} query.
+ *   </li>
+ *   <li>
+ *     To suggest across all contexts with the same boost,
+ *     use '*' as the context in {@link #addContext(CharSequence)})}.
+ *     This can be combined with specific contexts with different boosts.
+ *   </li>
+ *   <li>
+ *     To apply the same boost to multiple contexts sharing the same prefix,
+ *     Use {@link #addContext(CharSequence, float, boolean)} with the common
+ *     context prefix, boost and set <code>exact</code> to false.
+ *   <li>
+ *     Using this query against a {@link SuggestField} (not context enabled),
+ *     would yield results ignoring any context filtering/boosting
+ *   </li>
+ * </ul>
+ *
+ * @lucene.experimental
+ */
+public class ContextQuery extends CompletionQuery {
+  private Map<CharSequence, ContextMetaData> contexts;
+  /** Inner completion query */
+  protected CompletionQuery query;
+
+  /**
+   * Constructs a context completion query that matches
+   * documents specified by <code>query</code>.
+   * <p>
+   * Use {@link #addContext(CharSequence, float, boolean)}
+   * to add context(s) with boost
+   */
+  public ContextQuery(CompletionQuery query) {
+    super(query.getTerm(), query.getFilter());
+    if (query instanceof ContextQuery) {
+      throw new IllegalArgumentException("'query' parameter must not be of type "
+              + this.getClass().getSimpleName());
+    }
+    this.query = query;
+    contexts = new HashMap<>();
+  }
+
+  /**
+   * Adds an exact context with default boost of 1
+   */
+  public void addContext(CharSequence context) {
+    addContext(context, 1f, true);
+  }
+
+  /**
+   * Adds an exact context with boost
+   */
+  public void addContext(CharSequence context, float boost) {
+    addContext(context, boost, true);
+  }
+
+  /**
+   * Adds a context with boost, set <code>exact</code> to false
+   * if the context is a prefix of any indexed contexts
+   */
+  public void addContext(CharSequence context, float boost, boolean exact) {
+    if (boost < 0f) {
+      throw new IllegalArgumentException("'boost' must be >= 0");
+    }
+    for (int i = 0; i < context.length(); i++) {
+      if (ContextSuggestField.CONTEXT_SEPARATOR == context.charAt(i)) {
+        throw new IllegalArgumentException("Illegal value [" + context + "] UTF-16 codepoint [0x"
+            + Integer.toHexString((int) context.charAt(i))+ "] at position " + i + " is a reserved character");
+      }
+    }
+    contexts.put(context, new ContextMetaData(boost, exact));
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder buffer = new StringBuilder();
+    for (CharSequence context : contexts.keySet()) {
+      if (buffer.length() != 0) {
+        buffer.append(",");
+      } else {
+        buffer.append("contexts");
+        buffer.append(":[");
+      }
+      buffer.append(context);
+      ContextMetaData metaData = contexts.get(context);
+      if (metaData.exact == false) {
+        buffer.append("*");
+      }
+      if (metaData.boost != 0) {
+        buffer.append("^");
+        buffer.append(Float.toString(metaData.boost));
+      }
+    }
+    if (buffer.length() != 0) {
+      buffer.append("]");
+      buffer.append(",");
+    }
+    return buffer.toString() + query.toString(field);
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    IntsRefBuilder scratch = new IntsRefBuilder();
+    final Map<IntsRef, Float> contextMap = new HashMap<>(contexts.size());
+    final TreeSet<Integer> contextLengths = new TreeSet<>();
+    final CompletionWeight innerWeight = ((CompletionWeight) query.createWeight(searcher, needsScores));
+    Automaton contextsAutomaton = null;
+    Automaton gap = Automata.makeChar(ContextSuggestField.CONTEXT_SEPARATOR);
+    // if separators are preserved the fst contains a SEP_LABEL
+    // behind each gap. To have a matching automaton, we need to
+    // include the SEP_LABEL in the query as well
+    gap = Operations.concatenate(gap, Operations.optional(Automata.makeChar(CompletionAnalyzer.SEP_LABEL)));
+    final Automaton prefixAutomaton = Operations.concatenate(gap, innerWeight.getAutomaton());
+    final Automaton matchAllAutomaton = new RegExp(".*").toAutomaton();
+    for (Map.Entry<CharSequence, ContextMetaData> entry : contexts.entrySet()) {
+      Automaton contextAutomaton;
+      if (entry.getKey().equals("*")) {
+        contextAutomaton = Operations.concatenate(matchAllAutomaton, prefixAutomaton);
+      } else {
+        BytesRef ref = new BytesRef(entry.getKey());
+        ContextMetaData contextMetaData = entry.getValue();
+        contextMap.put(IntsRef.deepCopyOf(Util.toIntsRef(ref, scratch)), contextMetaData.boost);
+        contextLengths.add(scratch.length());
+        contextAutomaton = Automata.makeString(entry.getKey().toString());
+        if (contextMetaData.exact) {
+          contextAutomaton = Operations.concatenate(contextAutomaton, prefixAutomaton);
+        } else {
+          contextAutomaton = Operations.concatenate(Arrays.asList(contextAutomaton,
+              matchAllAutomaton,
+              prefixAutomaton));
+        }
+      }
+      if (contextsAutomaton == null) {
+        contextsAutomaton = contextAutomaton;
+      } else {
+        contextsAutomaton = Operations.union(contextsAutomaton, contextAutomaton);
+      }
+    }
+    if (contexts.size() == 0) {
+      addContext("*");
+      contextsAutomaton = Operations.concatenate(matchAllAutomaton, prefixAutomaton);
+    }
+    contextsAutomaton = Operations.determinize(contextsAutomaton, Operations.DEFAULT_MAX_DETERMINIZED_STATES);
+    int[] contextLengthArray = new int[contextLengths.size()];
+    final Iterator<Integer> iterator = contextLengths.descendingIterator();
+    for (int i = 0; iterator.hasNext(); i++) {
+      contextLengthArray[i] = iterator.next();
+    }
+    return new ContextCompletionWeight(searcher.getIndexReader(), this, contextsAutomaton,
+        innerWeight, contextMap, contextLengthArray);
+  }
+
+  private static class ContextMetaData {
+    private final float boost;
+    private final boolean exact;
+
+    private ContextMetaData(float boost, boolean exact) {
+      this.boost = boost;
+      this.exact = exact;
+    }
+  }
+
+  private class ContextCompletionWeight extends CompletionWeight {
+
+    private final Map<IntsRef, Float> contextMap;
+    private final int[] contextLengths;
+    private final CompletionWeight innerWeight;
+    private final BytesRefBuilder scratch = new BytesRefBuilder();
+
+    private float currentBoost;
+    private CharSequence currentContext;
+
+    public ContextCompletionWeight(IndexReader reader, CompletionQuery query,
+                                   Automaton automaton, CompletionWeight innerWeight,
+                                   Map<IntsRef, Float> contextMap,
+                                   int[] contextLengths) throws IOException {
+      super(reader, query, automaton);
+      this.contextMap = contextMap;
+      this.contextLengths = contextLengths;
+      this.innerWeight = innerWeight;
+    }
+
+    @Override
+    protected void setNextMatch(IntsRef pathPrefix) {
+      IntsRef ref = pathPrefix.clone();
+
+      // check if the pathPrefix matches any
+      // defined context, longer context first
+      for (int contextLength : contextLengths) {
+        if (contextLength > pathPrefix.length) {
+          continue;
+        }
+        ref.length = contextLength;
+        if (contextMap.containsKey(ref)) {
+          currentBoost = contextMap.get(ref);
+          ref.length = pathPrefix.length;
+          ref.offset = contextLength;
+          while (ref.ints[ref.offset] != ContextSuggestField.CONTEXT_SEPARATOR) {
+            ref.offset++;
+            assert ref.offset < ref.length;
+          }
+          assert ref.ints[ref.offset] == ContextSuggestField.CONTEXT_SEPARATOR : "expected CONTEXT_SEPARATOR at offset=" + ref.offset;
+          if (ref.offset > pathPrefix.offset) {
+            currentContext = Util.toBytesRef(new IntsRef(pathPrefix.ints, pathPrefix.offset, ref.offset), scratch).utf8ToString();
+          } else {
+            currentContext = null;
+          }
+          ref.offset++;
+          if (ref.ints[ref.offset] == CompletionAnalyzer.SEP_LABEL) {
+            ref.offset++;
+          }
+          innerWeight.setNextMatch(ref);
+          return;
+        }
+      }
+      // unknown context
+      ref.length = pathPrefix.length;
+      currentBoost = contexts.get("*").boost;
+      for (int i = pathPrefix.offset; i < pathPrefix.length; i++) {
+        if (pathPrefix.ints[i] == ContextSuggestField.CONTEXT_SEPARATOR) {
+          if (i > pathPrefix.offset) {
+            currentContext = Util.toBytesRef(new IntsRef(pathPrefix.ints, pathPrefix.offset, i), scratch).utf8ToString();
+          } else {
+            currentContext = null;
+          }
+          ref.offset = ++i;
+          assert ref.offset < ref.length : "input should not end with the context separator";
+          if (pathPrefix.ints[i] == CompletionAnalyzer.SEP_LABEL) {
+            ref.offset++;
+            assert ref.offset < ref.length : "input should not end with a context separator followed by SEP_LABEL";
+          }
+          ref.length -= ref.offset;
+          innerWeight.setNextMatch(ref);
+        }
+      }
+    }
+
+    @Override
+    protected CharSequence context() {
+      return currentContext;
+    }
+
+    @Override
+    protected float boost() {
+      return currentBoost + innerWeight.boost();
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextSuggestField.java Thu May 28 07:53:09 2015
@@ -0,0 +1,167 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+
+/**
+ * {@link SuggestField} which additionally takes in a set of
+ * contexts. Example usage of adding a suggestion with contexts is as follows:
+ *
+ * <pre class="prettyprint">
+ *  document.add(
+ *   new ContextSuggestField(name, "suggestion", Arrays.asList("context1", "context2"),  4));
+ * </pre>
+ *
+ * Use {@link ContextQuery} to boost and/or filter suggestions
+ * at query-time. Use {@link PrefixCompletionQuery}, {@link RegexCompletionQuery}
+ * or {@link FuzzyCompletionQuery} if context boost/filtering
+ * are not needed.
+ *
+ * @lucene.experimental
+ */
+public class ContextSuggestField extends SuggestField {
+
+  /**
+   * Separator used between context value and the suggest field value
+   */
+  public static final int CONTEXT_SEPARATOR = '\u001D';
+  static final byte TYPE = 1;
+
+  private final Set<CharSequence> contexts;
+
+  /**
+   * Creates a context-enabled suggest field
+   *
+   * @param name field name
+   * @param contexts associated contexts
+   * @param value field value to get suggestion 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 or
+   * contexts contains any reserved characters
+   */
+  public ContextSuggestField(String name, Collection<CharSequence> contexts, String value, int weight) {
+    super(name, value, weight);
+    validate(value);
+    if (contexts != null) {
+      for (CharSequence context : contexts) {
+        validate(context);
+      }
+      this.contexts = new HashSet<>(contexts);
+    } else {
+      this.contexts = new HashSet<>();
+    }
+  }
+
+  @Override
+  protected CompletionTokenStream wrapTokenStream(TokenStream stream) {
+    CompletionTokenStream completionTokenStream;
+    if (stream instanceof CompletionTokenStream) {
+      completionTokenStream = (CompletionTokenStream) stream;
+      completionTokenStream = new CompletionTokenStream(
+          new PrefixTokenFilter(stream, (char) CONTEXT_SEPARATOR, contexts),
+          completionTokenStream.preserveSep,
+          completionTokenStream.preservePositionIncrements,
+          completionTokenStream.maxGraphExpansions);
+    } else {
+      completionTokenStream = new CompletionTokenStream(
+          new PrefixTokenFilter(stream, (char) CONTEXT_SEPARATOR, contexts));
+    }
+    return completionTokenStream;
+  }
+
+  @Override
+  protected byte type() {
+    return TYPE;
+  }
+
+  /**
+   * The {@link PrefixTokenFilter} wraps a {@link TokenStream} and adds a set
+   * prefixes ahead. The position attribute will not be incremented for the prefixes.
+   */
+  private static final class PrefixTokenFilter extends TokenFilter {
+
+    private final char separator;
+    private final CharTermAttribute termAttr = addAttribute(CharTermAttribute.class);
+    private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
+    private final Iterable<CharSequence> prefixes;
+
+    private Iterator<CharSequence> currentPrefix;
+
+    /**
+     * Create a new {@link PrefixTokenFilter}
+     *
+     * @param input {@link TokenStream} to wrap
+     * @param separator Character used separate prefixes from other tokens
+     * @param prefixes {@link Iterable} of {@link CharSequence} which keeps all prefixes
+     */
+    public PrefixTokenFilter(TokenStream input, char separator, Iterable<CharSequence> prefixes) {
+      super(input);
+      this.prefixes = prefixes;
+      this.currentPrefix = null;
+      this.separator = separator;
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (currentPrefix != null) {
+        if (!currentPrefix.hasNext()) {
+          return input.incrementToken();
+        } else {
+          posAttr.setPositionIncrement(0);
+        }
+      } else {
+        currentPrefix = prefixes.iterator();
+        termAttr.setEmpty();
+        posAttr.setPositionIncrement(1);
+      }
+      termAttr.setEmpty();
+      if (currentPrefix.hasNext()) {
+        termAttr.append(currentPrefix.next());
+      }
+      termAttr.append(separator);
+      return true;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      currentPrefix = null;
+    }
+  }
+
+  private void validate(final CharSequence value) {
+    for (int i = 0; i < value.length(); i++) {
+      if (CONTEXT_SEPARATOR == value.charAt(i)) {
+        throw new IllegalArgumentException("Illegal value [" + value + "] UTF-16 codepoint [0x"
+            + Integer.toHexString((int) value.charAt(i))+ "] at position " + i + " is a reserved character");
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/FuzzyCompletionQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,254 @@
+package org.apache.lucene.search.suggest.document;
+
+/*
+ * 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.Arrays;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.LevenshteinAutomata;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
+
+/**
+ * A {@link CompletionQuery} that match documents containing terms
+ * within an edit distance of the specified prefix.
+ * <p>
+ * This query boost documents relative to how similar the indexed terms are to the
+ * provided prefix.
+ * <p>
+ * Example usage of querying an analyzed prefix within an edit distance of 1 of 'subg'
+ * against a field 'suggest_field' is as follows:
+ *
+ * <pre class="prettyprint">
+ *  CompletionQuery query = new FuzzyCompletionQuery(analyzer, new Term("suggest_field", "subg"));
+ * </pre>
+ *
+ * @lucene.experimental
+ */
+public class FuzzyCompletionQuery extends PrefixCompletionQuery {
+
+  /**
+   * Measure maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix
+   * parameters in Unicode code points (actual letters)
+   * instead of bytes.
+   * */
+  public static final boolean DEFAULT_UNICODE_AWARE = false;
+
+  /**
+   * The default minimum length of the key before any edits are allowed.
+   */
+  public static final int DEFAULT_MIN_FUZZY_LENGTH = 3;
+
+  /**
+   * The default prefix length where edits are not allowed.
+   */
+  public static final int DEFAULT_NON_FUZZY_PREFIX = 1;
+
+  /**
+   * The default maximum number of edits for fuzzy
+   * suggestions.
+   */
+  public static final int DEFAULT_MAX_EDITS = 1;
+
+  /**
+   * The default transposition value passed to {@link LevenshteinAutomata}
+   */
+  public static final boolean DEFAULT_TRANSPOSITIONS = true;
+
+  private final int maxEdits;
+  private final boolean transpositions;
+  private final int nonFuzzyPrefix;
+  private final int minFuzzyLength;
+  private final boolean unicodeAware;
+  private final int maxDeterminizedStates;
+
+  /**
+   * Calls {@link FuzzyCompletionQuery#FuzzyCompletionQuery(Analyzer, Term, Filter)}
+   * with no filter
+   */
+  public FuzzyCompletionQuery(Analyzer analyzer, Term term) {
+    this(analyzer, term, null);
+  }
+
+  /**
+   * Calls {@link FuzzyCompletionQuery#FuzzyCompletionQuery(Analyzer, Term, Filter,
+   * int, boolean, int, int, boolean, int)}
+   * with defaults for <code>maxEdits</code>, <code>transpositions</code>,
+   * <code>nonFuzzyPrefix</code>, <code>minFuzzyLength</code>,
+   * <code>unicodeAware</code> and <code>maxDeterminizedStates</code>
+   *
+   * See {@link #DEFAULT_MAX_EDITS}, {@link #DEFAULT_TRANSPOSITIONS},
+   * {@link #DEFAULT_NON_FUZZY_PREFIX}, {@link #DEFAULT_MIN_FUZZY_LENGTH},
+   * {@link #DEFAULT_UNICODE_AWARE} and {@link Operations#DEFAULT_MAX_DETERMINIZED_STATES}
+   * for defaults
+   */
+  public FuzzyCompletionQuery(Analyzer analyzer, Term term, Filter filter) {
+    this(analyzer, term, filter, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS, DEFAULT_NON_FUZZY_PREFIX,
+        DEFAULT_MIN_FUZZY_LENGTH, DEFAULT_UNICODE_AWARE, Operations.DEFAULT_MAX_DETERMINIZED_STATES
+    );
+  }
+
+  /**
+   * Constructs an analyzed fuzzy prefix completion query
+   *
+   * @param analyzer used to analyze the provided {@link Term#text()}
+   * @param term query is run against {@link Term#field()} and {@link Term#text()}
+   *             is analyzed with <code>analyzer</code>
+   * @param filter used to query on a sub set of documents
+   * @param maxEdits maximum number of acceptable edits
+   * @param transpositions value passed to {@link LevenshteinAutomata}
+   * @param nonFuzzyPrefix prefix length where edits are not allowed
+   * @param minFuzzyLength minimum prefix length before any edits are allowed
+   * @param unicodeAware treat prefix as unicode rather than bytes
+   * @param maxDeterminizedStates maximum automaton states allowed for {@link LevenshteinAutomata}
+   */
+  public FuzzyCompletionQuery(Analyzer analyzer, Term term, Filter filter, int maxEdits,
+                              boolean transpositions, int nonFuzzyPrefix, int minFuzzyLength,
+                              boolean unicodeAware, int maxDeterminizedStates) {
+    super(analyzer, term, filter);
+    this.maxEdits = maxEdits;
+    this.transpositions = transpositions;
+    this.nonFuzzyPrefix = nonFuzzyPrefix;
+    this.minFuzzyLength = minFuzzyLength;
+    this.unicodeAware = unicodeAware;
+    this.maxDeterminizedStates = maxDeterminizedStates;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
+    Automaton a = stream.toAutomaton(unicodeAware);
+    final Set<IntsRef> refs = Operations.getFiniteStrings(a, -1);
+    assert refs.size() > 0;
+    Automaton automaton = toLevenshteinAutomata(refs);
+    if (unicodeAware) {
+      Automaton utf8automaton = new UTF32ToUTF8().convert(automaton);
+      utf8automaton = Operations.determinize(utf8automaton, maxDeterminizedStates);
+      automaton = utf8automaton;
+    }
+    return new FuzzyCompletionWeight(searcher.getIndexReader(), this, automaton, refs);
+  }
+
+  private Automaton toLevenshteinAutomata(Set<IntsRef> ref) {
+    Automaton subs[] = new Automaton[ref.size()];
+    int upto = 0;
+    for (IntsRef path : ref) {
+      if (path.length <= nonFuzzyPrefix || path.length < minFuzzyLength) {
+        subs[upto] = Automata.makeString(path.ints, path.offset, path.length);
+        upto++;
+      } else {
+        int ints[] = new int[path.length - nonFuzzyPrefix];
+        System.arraycopy(path.ints, path.offset + nonFuzzyPrefix, ints, 0, ints.length);
+        // TODO: maybe add alphaMin to LevenshteinAutomata,
+        // and pass 1 instead of 0?  We probably don't want
+        // to allow the trailing dedup bytes to be
+        // edited... but then 0 byte is "in general" allowed
+        // on input (but not in UTF8).
+        LevenshteinAutomata lev = new LevenshteinAutomata(ints,
+            unicodeAware ? Character.MAX_CODE_POINT : 255,
+            transpositions);
+        subs[upto] = lev.toAutomaton(maxEdits,
+            UnicodeUtil.newString(path.ints, path.offset, nonFuzzyPrefix));
+        upto++;
+      }
+    }
+
+    if (subs.length == 0) {
+      // automaton is empty, there is no accepted paths through it
+      return Automata.makeEmpty(); // matches nothing
+    } else if (subs.length == 1) {
+      // no synonyms or anything: just a single path through the tokenstream
+      return subs[0];
+    } else {
+      // multiple paths: this is really scary! is it slow?
+      // maybe we should not do this and throw UOE?
+      Automaton a = Operations.union(Arrays.asList(subs));
+      // TODO: we could call toLevenshteinAutomata() before det?
+      // this only happens if you have multiple paths anyway (e.g. synonyms)
+      return Operations.determinize(a, maxDeterminizedStates);
+    }
+  }
+
+  @Override
+  public String toString(String field) {
+    StringBuilder buffer = new StringBuilder();
+    if (!getField().equals(field)) {
+      buffer.append(getField());
+      buffer.append(":");
+    }
+    buffer.append(getTerm().text());
+    buffer.append('*');
+    buffer.append('~');
+    buffer.append(Integer.toString(maxEdits));
+    if (getFilter() != null) {
+      buffer.append(",");
+      buffer.append("filter");
+      buffer.append(getFilter().toString(field));
+    }
+    return buffer.toString();
+  }
+
+  private static class FuzzyCompletionWeight extends CompletionWeight {
+    private final Set<IntsRef> refs;
+    int currentBoost = 0;
+
+    public FuzzyCompletionWeight(IndexReader reader, CompletionQuery query,
+                                 Automaton automaton, Set<IntsRef> refs) throws IOException {
+      super(reader, query, automaton);
+      this.refs = refs;
+    }
+
+    @Override
+    protected void setNextMatch(IntsRef pathPrefix) {
+      // NOTE: the last letter of the matched prefix for the exact
+      // match never makes it through here
+      // so an exact match and a match with only a edit at the
+      // end is boosted the same
+      int maxCount = 0;
+      for (IntsRef ref : refs) {
+        int minLength = Math.min(ref.length, pathPrefix.length);
+        int count = 0;
+        for (int i = 0; i < minLength; i++) {
+          if (ref.ints[i + ref.offset] == pathPrefix.ints[i + pathPrefix.offset]) {
+            count++;
+          } else {
+            break;
+          }
+        }
+        maxCount = Math.max(maxCount, count);
+      }
+      currentBoost = maxCount;
+    }
+
+    @Override
+    protected float boost() {
+      return currentBoost;
+    }
+  }
+}