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 [2/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/

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

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/PrefixCompletionQuery.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.analysis.Analyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
+
+/**
+ * A {@link CompletionQuery} which takes an {@link Analyzer}
+ * to analyze the prefix of the query term.
+ * <p>
+ * Example usage of querying an analyzed prefix 'sugg'
+ * against a field 'suggest_field' is as follows:
+ *
+ * <pre class="prettyprint">
+ *  CompletionQuery query = new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg"));
+ * </pre>
+ * @lucene.experimental
+ */
+public class PrefixCompletionQuery extends CompletionQuery {
+  /** Used to analyze the term text */
+  protected final CompletionAnalyzer analyzer;
+
+  /**
+   * Calls {@link PrefixCompletionQuery#PrefixCompletionQuery(Analyzer, Term, Filter)}
+   * with no filter
+   */
+  public PrefixCompletionQuery(Analyzer analyzer, Term term) {
+    this(analyzer, term, null);
+  }
+
+  /**
+   * Constructs an analyzed 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
+   */
+  public PrefixCompletionQuery(Analyzer analyzer, Term term, Filter filter) {
+    super(term, filter);
+    if (!(analyzer instanceof CompletionAnalyzer)) {
+      this.analyzer = new CompletionAnalyzer(analyzer);
+    } else {
+      this.analyzer = (CompletionAnalyzer) analyzer;
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    CompletionTokenStream stream = (CompletionTokenStream) analyzer.tokenStream(getField(), getTerm().text());
+    return new CompletionWeight(searcher.getIndexReader(), this, stream.toAutomaton());
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/RegexCompletionQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,96 @@
+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.Term;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.RegExp;
+
+/**
+ * A {@link CompletionQuery} which takes a regular expression
+ * as the prefix of the query term.
+ *
+ * <p>
+ * Example usage of querying a prefix of 'sug' and 'sub'
+ * as a regular expression against a suggest field 'suggest_field':
+ *
+ * <pre class="prettyprint">
+ *  CompletionQuery query = new RegexCompletionQuery(new Term("suggest_field", "su[g|b]"));
+ * </pre>
+ *
+ * <p>
+ * See {@link RegExp} for the supported regular expression
+ * syntax
+ *
+ * @lucene.experimental
+ */
+public class RegexCompletionQuery extends CompletionQuery {
+
+  private final int flags;
+  private final int maxDeterminizedStates;
+
+  /**
+   * Calls {@link RegexCompletionQuery#RegexCompletionQuery(Term, Filter)}
+   * with no filter
+   */
+  public RegexCompletionQuery(Term term) {
+    this(term, null);
+  }
+
+  /**
+   * Calls {@link RegexCompletionQuery#RegexCompletionQuery(Term, int, int, Filter)}
+   * enabling all optional regex syntax and <code>maxDeterminizedStates</code> of
+   * {@value Operations#DEFAULT_MAX_DETERMINIZED_STATES}
+   */
+  public RegexCompletionQuery(Term term, Filter filter) {
+    this(term, RegExp.ALL, Operations.DEFAULT_MAX_DETERMINIZED_STATES, filter);
+  }
+  /**
+   * Calls {@link RegexCompletionQuery#RegexCompletionQuery(Term, int, int, Filter)}
+   * with no filter
+   */
+  public RegexCompletionQuery(Term term, int flags, int maxDeterminizedStates) {
+    this(term, flags, maxDeterminizedStates, null);
+  }
+
+  /**
+   * Constructs a regular expression completion query
+   *
+   * @param term query is run against {@link Term#field()} and {@link Term#text()}
+   *             is interpreted as a regular expression
+   * @param flags used as syntax_flag in {@link RegExp#RegExp(String, int)}
+   * @param maxDeterminizedStates used in {@link RegExp#toAutomaton(int)}
+   * @param filter used to query on a sub set of documents
+   */
+  public RegexCompletionQuery(Term term, int flags, int maxDeterminizedStates, Filter filter) {
+    super(term, filter);
+    this.flags = flags;
+    this.maxDeterminizedStates = maxDeterminizedStates;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new CompletionWeight(searcher.getIndexReader(), this,
+        new RegExp(getTerm().text(), flags).toAutomaton(maxDeterminizedStates));
+  }
+}

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

Added: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,522 @@
+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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.lucene.search.suggest.document.TestSuggestField.Entry;
+import static org.apache.lucene.search.suggest.document.TestSuggestField.assertSuggestions;
+import static org.apache.lucene.search.suggest.document.TestSuggestField.iwcWithSuggestField;
+
+public class TestContextQuery extends LuceneTestCase {
+  public Directory dir;
+
+  @Before
+  public void before() throws Exception {
+    dir = newDirectory();
+  }
+
+  @After
+  public void after() throws Exception {
+    dir.close();
+  }
+
+  @Test
+  public void testIllegalInnerQuery() throws Exception {
+    try {
+      new ContextQuery(new ContextQuery(
+          new PrefixCompletionQuery(new MockAnalyzer(random()), new Term("suggest_field", "sugg"))));
+      fail("should error out trying to nest a Context query within another Context query");
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains(ContextQuery.class.getSimpleName()));
+    }
+  }
+
+  @Test
+  public void testSimpleContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 5));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type1", 1);
+    query.addContext("type2", 2);
+    query.addContext("type3", 3);
+    query.addContext("type4", 4);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion4", "type4", 5 * 4),
+        new Entry("suggestion3", "type3", 6 * 3),
+        new Entry("suggestion2", "type2", 7 * 2),
+        new Entry("suggestion1", "type1", 8 * 1)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testContextQueryOnSuggestField() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    Document document = new Document();
+
+    document.add(new SuggestField("suggest_field", "abc", 3));
+    document.add(new SuggestField("suggest_field", "abd", 4));
+    document.add(new SuggestField("suggest_field", "The Foo Fighters", 2));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new SuggestField("suggest_field", "abcdd", 5));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "ab")));
+    try {
+      suggestIndexSearcher.suggest(query, 4);
+    } catch (IllegalStateException expected) {
+      assertTrue(expected.getMessage().contains("SuggestField"));
+    }
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testNonExactContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type", 1, false);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "type1", 4),
+        new Entry("suggestion2", "type2", 3),
+        new Entry("suggestion3", "type3", 2),
+        new Entry("suggestion4", "type4", 1));
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testContextPrecedenceBoost() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type", 1);
+    query.addContext("typetype", 2);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "typetype", 4 * 2),
+        new Entry("suggestion2", "type", 3 * 1)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testEmptyContext() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    Document document = new Document();
+
+    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));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion_no_ctx", null, 4),
+        new Entry("suggestion", "type4", 1));
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testEmptyContextWithBoosts() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    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", null, "suggestion3", 2));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type4", 10);
+    query.addContext("*");
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion4", "type4", 1 * 10),
+        new Entry("suggestion1", null, 4),
+        new Entry("suggestion2", null, 3),
+        new Entry("suggestion3", null, 2)
+    );
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testSameSuggestionMultipleContext() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type1", 10);
+    query.addContext("type2", 2);
+    query.addContext("type3", 3);
+    query.addContext("type4", 4);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion", "type1", 4 * 10),
+        new Entry("suggestion", "type3", 4 * 3),
+        new Entry("suggestion", "type2", 4 * 2),
+        new Entry("suggestion", "type4", 1 * 4)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testMixedContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type1", 7);
+    query.addContext("type2", 6);
+    query.addContext("*", 5);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "type1", 4 * 7),
+        new Entry("suggestion2", "type2", 3 * 6),
+        new Entry("suggestion3", "type3", 2 * 5),
+        new Entry("suggestion4", "type4", 1 * 5)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testFilteringContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type3", 3);
+    query.addContext("type4", 4);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion3", "type3", 2 * 3),
+        new Entry("suggestion4", "type4", 1 * 4)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testContextQueryRewrite() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    CompletionQuery query = new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg"));
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "type1", 4),
+        new Entry("suggestion2", "type2", 3),
+        new Entry("suggestion3", "type3", 2),
+        new Entry("suggestion4", "type4", 1));
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testMultiContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 5));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    query.addContext("type1", 1);
+    query.addContext("type2", 2);
+    query.addContext("type3", 3);
+    query.addContext("type4", 4);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "type3", 8 * 3),
+        new Entry("suggestion4", "type4", 5 * 4),
+        new Entry("suggestion3", "type3", 6 * 3),
+        new Entry("suggestion2", "type2", 7 * 2),
+        new Entry("suggestion1", "type1", 8 * 1));
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testAllContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 4);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "type1", 4),
+        new Entry("suggestion2", "type2", 3),
+        new Entry("suggestion3", "type3", 2),
+        new Entry("suggestion4", "type4", 1));
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testRandomContextQueryScoring() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    int numSuggestions = atLeast(20);
+    int numContexts = atLeast(5);
+
+    Set<Integer> seenWeights = new HashSet<>();
+    List<Entry> expectedEntries = new ArrayList<>();
+    List<CharSequence> contexts = new ArrayList<>();
+    for (int i = 1; i <= numContexts; i++) {
+      CharSequence context = TestUtil.randomSimpleString(random(), 10) + i;
+      contexts.add(context);
+      for (int j = 1; j <= numSuggestions; j++) {
+        String suggestion = "sugg_" + TestUtil.randomSimpleString(random(), 10) + j;
+        int weight = TestUtil.nextInt(random(), 1, 1000 * numContexts * numSuggestions);
+        while (seenWeights.contains(weight)) {
+          weight = TestUtil.nextInt(random(), 1, 1000 * numContexts * numSuggestions);
+        }
+        seenWeights.add(weight);
+        Document document = new Document();
+        document.add(new ContextSuggestField("suggest_field", Collections.singletonList(context), suggestion, weight));
+        iw.addDocument(document);
+        expectedEntries.add(new Entry(suggestion, context.toString(), i * weight));
+      }
+      if (rarely()) {
+        iw.commit();
+      }
+    }
+    Entry[] expectedResults = expectedEntries.toArray(new Entry[expectedEntries.size()]);
+
+    ArrayUtil.introSort(expectedResults, new Comparator<Entry>() {
+      @Override
+      public int compare(Entry o1, Entry o2) {
+        int cmp = Float.compare(o2.value, o1.value);
+        if (cmp != 0) {
+          return cmp;
+        } else {
+          return o1.output.compareTo(o2.output);
+        }
+      }
+    });
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    ContextQuery query = new ContextQuery(new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg")));
+    for (int i = 0; i < contexts.size(); i++) {
+      query.addContext(contexts.get(i), i + 1);
+    }
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 4);
+    assertSuggestions(suggest, Arrays.copyOfRange(expectedResults, 0, 4));
+
+    reader.close();
+    iw.close();
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestContextSuggestField.java Thu May 28 07:53:09 2015
@@ -0,0 +1,145 @@
+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.util.Collections;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.CharsRefBuilder;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.lucene.search.suggest.document.TestSuggestField.Entry;
+import static org.apache.lucene.search.suggest.document.TestSuggestField.assertSuggestions;
+import static org.apache.lucene.search.suggest.document.TestSuggestField.iwcWithSuggestField;
+
+public class TestContextSuggestField extends LuceneTestCase {
+
+  public Directory dir;
+
+  @Before
+  public void before() throws Exception {
+    dir = newDirectory();
+  }
+
+  @After
+  public void after() throws Exception {
+    dir.close();
+  }
+
+  @Test
+  public void testEmptySuggestion() throws Exception {
+    try {
+      new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "", 1);
+      fail("no exception thrown when indexing zero length suggestion");
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains("value"));
+    }
+  }
+
+  @Test
+  public void testReservedChars() throws Exception {
+    CharsRefBuilder charsRefBuilder = new CharsRefBuilder();
+    charsRefBuilder.append("sugg");
+    charsRefBuilder.setCharAt(2, (char) ContextSuggestField.CONTEXT_SEPARATOR);
+    try {
+      new ContextSuggestField("name", Collections.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);
+      fail("no exception thrown for value containing CONTEXT_SEPARATOR:" + ContextSuggestField.CONTEXT_SEPARATOR);
+    } catch (IllegalArgumentException e) {
+      assertTrue(e.getMessage().contains("[0x1d]"));
+    }
+  }
+
+  @Test
+  public void testMixedSuggestFields() throws Exception {
+    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));
+
+    try (RandomIndexWriter iw = new RandomIndexWriter(random(), dir,
+        iwcWithSuggestField(analyzer, "suggest_field"))) {
+      iw.addDocument(document);
+      iw.commit();
+      fail("mixing suggest field types for same field name should error out");
+    } catch (IllegalArgumentException expected) {
+      assertTrue(expected.getMessage().contains("mixed types"));
+    }
+  }
+
+  @Test
+  public void testWithSuggestFields() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir,
+        iwcWithSuggestField(analyzer, "suggest_field", "context_suggest_field"));
+    Document document = new Document();
+
+    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));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new SuggestField("suggest_field", "suggestion4", 1));
+    document.add(new ContextSuggestField("context_suggest_field", Collections.singletonList("type4"), "suggestion4", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+
+    CompletionQuery query = new PrefixCompletionQuery(analyzer, new Term("suggest_field", "sugg"));
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 10);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", 4),
+        new Entry("suggestion2", 3),
+        new Entry("suggestion3", 2),
+        new Entry("suggestion4", 1));
+
+    query = new PrefixCompletionQuery(analyzer, new Term("context_suggest_field", "sugg"));
+    suggest = suggestIndexSearcher.suggest(query, 10);
+    assertSuggestions(suggest,
+        new Entry("suggestion1", "type1", 4),
+        new Entry("suggestion2", "type2", 3),
+        new Entry("suggestion3", "type3", 2),
+        new Entry("suggestion4", "type4", 1));
+
+    reader.close();
+    iw.close();
+  }
+}

Added: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java?rev=1682158&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java (added)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/document/TestFuzzyCompletionQuery.java Thu May 28 07:53:09 2015
@@ -0,0 +1,153 @@
+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.util.Collections;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.lucene.search.suggest.document.TestSuggestField.Entry;
+import static org.apache.lucene.search.suggest.document.TestSuggestField.assertSuggestions;
+import static org.apache.lucene.search.suggest.document.TestSuggestField.iwcWithSuggestField;
+
+public class TestFuzzyCompletionQuery extends LuceneTestCase {
+  public Directory dir;
+
+  @Before
+  public void before() throws Exception {
+    dir = newDirectory();
+  }
+
+  @After
+  public void after() throws Exception {
+    dir.close();
+  }
+
+  @Test
+  public void testFuzzyQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    Document document = new Document();
+
+    document.add(new SuggestField("suggest_field", "suggestion", 2));
+    document.add(new SuggestField("suggest_field", "suaggestion", 4));
+    document.add(new SuggestField("suggest_field", "ssuggestion", 1));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new SuggestField("suggest_field", "sugfoo", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    CompletionQuery query = new FuzzyCompletionQuery(analyzer, new Term("suggest_field", "sugg"));
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 4);
+    assertSuggestions(suggest,
+        new Entry("suaggestion", 4 * 2),
+        new Entry("suggestion", 2 * 3),
+        new Entry("sugfoo", 1 * 3),
+        new Entry("ssuggestion", 1 * 1)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testFuzzyContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    Document document = new Document();
+
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "sduggestion", 1));
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "sudggestion", 1));
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "sugdgestion", 1));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggdestion", 1));
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    CompletionQuery query =  new ContextQuery(new FuzzyCompletionQuery(analyzer, new Term("suggest_field", "sugge")));
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(query, 5);
+    assertSuggestions(suggest,
+        new Entry("suggestion", "type4", 1 + 4),
+        new Entry("suggdestion", "type4", 1 + 4),
+        new Entry("sugdgestion", "type3", 1 + 3),
+        new Entry("sudggestion", "type2", 1 + 2),
+        new Entry("sduggestion", "type1", 1 + 1)
+    );
+
+    reader.close();
+    iw.close();
+  }
+
+  @Test
+  public void testFuzzyFilteredContextQuery() throws Exception {
+    Analyzer analyzer = new MockAnalyzer(random());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwcWithSuggestField(analyzer, "suggest_field"));
+    Document document = new Document();
+
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type1"), "sduggestion", 1));
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type2"), "sudggestion", 1));
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type3"), "sugdgestion", 1));
+    iw.addDocument(document);
+    document.clear();
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggdestion", 1));
+    document.add(new ContextSuggestField("suggest_field", Collections.singletonList("type4"), "suggestion", 1));
+    iw.addDocument(document);
+
+    if (rarely()) {
+      iw.commit();
+    }
+
+    DirectoryReader reader = iw.getReader();
+    SuggestIndexSearcher suggestIndexSearcher = new SuggestIndexSearcher(reader);
+    CompletionQuery fuzzyQuery = new FuzzyCompletionQuery(analyzer, new Term("suggest_field", "sugge"));
+    ContextQuery contextQuery = new ContextQuery(fuzzyQuery);
+    contextQuery.addContext("type1", 6);
+    contextQuery.addContext("type3", 2);
+    TopSuggestDocs suggest = suggestIndexSearcher.suggest(contextQuery, 5);
+    assertSuggestions(suggest,
+        new Entry("sduggestion", "type1", 1 * (1 + 6)),
+        new Entry("sugdgestion", "type3", 1 * (3 + 2))
+    );
+
+    reader.close();
+    iw.close();
+  }
+}