You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by sy...@apache.org on 2014/09/16 00:24:51 UTC

[4/8] Porting Lucene.Net.Suggest (still not compiling)

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/Analyzing/FSTUtil.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/Analyzing/FSTUtil.cs b/src/Lucene.Net.Suggest/Suggest/Analyzing/FSTUtil.cs
new file mode 100644
index 0000000..3ff019e
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/Analyzing/FSTUtil.cs
@@ -0,0 +1,146 @@
+using System.Collections.Generic;
+using System.Diagnostics;
+using Lucene.Net.Util;
+using Lucene.Net.Util.Automaton;
+using Lucene.Net.Util.Fst;
+
+namespace Lucene.Net.Search.Suggest.Analyzing
+{
+
+    /*
+     * 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.
+     */
+    // TODO: move to core?  nobody else uses it yet though...
+
+    /// <summary>
+    /// Exposes a utility method to enumerate all paths
+    /// intersecting an <seealso cref="Automaton"/> with an <seealso cref="FST"/>.
+    /// </summary>
+    public class FSTUtil
+    {
+
+        private FSTUtil()
+        {
+        }
+
+        /// <summary>
+        /// Holds a pair (automaton, fst) of states and accumulated output in the intersected machine. </summary>
+        public sealed class Path<T>
+        {
+
+            /// <summary>
+            /// Node in the automaton where path ends: </summary>
+            public readonly State state;
+
+            /// <summary>
+            /// Node in the FST where path ends: </summary>
+            public readonly FST.Arc<T> fstNode;
+
+            /// <summary>
+            /// Output of the path so far: </summary>
+            internal T output;
+
+            /// <summary>
+            /// Input of the path so far: </summary>
+            public readonly IntsRef input;
+
+            /// <summary>
+            /// Sole constructor. </summary>
+            public Path(State state, FST.Arc<T> fstNode, T output, IntsRef input)
+            {
+                this.state = state;
+                this.fstNode = fstNode;
+                this.output = output;
+                this.input = input;
+            }
+        }
+
+        /// <summary>
+        /// Enumerates all minimal prefix paths in the automaton that also intersect the FST,
+        /// accumulating the FST end node and output for each path.
+        /// </summary>
+        public static IList<Path<T>> IntersectPrefixPaths<T>(Automaton a, FST<T> fst)
+	  {
+		Debug.Assert(a.Deterministic);
+		IList<Path<T>> queue = new List<Path<T>>();
+		IList<Path<T>> endNodes = new List<Path<T>>();
+		queue.Add(new Path<>(a.InitialState, fst.GetFirstArc(new FST.Arc<T>()), fst.Outputs.NoOutput, new IntsRef()));
+
+		FST.Arc<T> scratchArc = new FST.Arc<T>();
+		FST.BytesReader fstReader = fst.BytesReader;
+
+		while (queue.Count != 0)
+		{
+		  Path<T> path = queue.RemoveAt(queue.Count - 1);
+		  if (path.state.Accept)
+		  {
+			endNodes.Add(path);
+			// we can stop here if we accept this path,
+			// we accept all further paths too
+			continue;
+		  }
+
+		  IntsRef currentInput = path.input;
+		  foreach (Transition t in path.state.Transitions)
+		  {
+			int min = t.Min;
+			int max = t.Max;
+			if (min == max)
+			{
+			  FST.Arc<T> nextArc = fst.FindTargetArc(t.Min, path.fstNode, scratchArc, fstReader);
+			  if (nextArc != null)
+			  {
+				IntsRef newInput = new IntsRef(currentInput.Length + 1);
+				newInput.CopyInts(currentInput);
+				newInput.Ints[currentInput.Length] = t.Min;
+				newInput.Length = currentInput.Length + 1;
+				queue.Add(new Path<>(t.Dest, new FST.Arc<T>()
+				  .CopyFrom(nextArc), fst.Outputs.Add(path.output, nextArc.Output), newInput));
+			  }
+			}
+			else
+			{
+			  // TODO: if this transition's TO state is accepting, and
+			  // it accepts the entire range possible in the FST (ie. 0 to 255),
+			  // we can simply use the prefix as the accepted state instead of
+			  // looking up all the ranges and terminate early
+			  // here.  This just shifts the work from one queue
+			  // (this one) to another (the completion search
+			  // done in AnalyzingSuggester).
+			  FST.Arc<T> nextArc = Util.ReadCeilArc(min, fst, path.fstNode, scratchArc, fstReader);
+			  while (nextArc != null && nextArc.Label <= max)
+			  {
+				Debug.Assert(nextArc.Label <= max);
+				Debug.Assert(nextArc.Label >= min, nextArc.Label + " " + min);
+				IntsRef newInput = new IntsRef(currentInput.Length + 1);
+				newInput.CopyInts(currentInput);
+				newInput.Ints[currentInput.Length] = nextArc.Label;
+				newInput.Length = currentInput.Length + 1;
+				queue.Add(new Path<>(t.Dest, new FST.Arc<T>()
+				  .CopyFrom(nextArc), fst.Outputs.Add(path.output, nextArc.Output), newInput));
+				int label = nextArc.Label; // used in assert
+				nextArc = nextArc.Last ? null : fst.ReadNextRealArc(nextArc, fstReader);
+				Debug.Assert(nextArc == null || label < nextArc.Label, "last: " + label + " next: " + nextArc.Label);
+			  }
+			}
+		  }
+		}
+		return endNodes;
+	  }
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/Analyzing/FreeTextSuggester.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/Analyzing/FreeTextSuggester.cs b/src/Lucene.Net.Suggest/Suggest/Analyzing/FreeTextSuggester.cs
new file mode 100644
index 0000000..4c82305
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/Analyzing/FreeTextSuggester.cs
@@ -0,0 +1,929 @@
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.IO;
+using Lucene.Net.Analysis;
+using Lucene.Net.Analysis.Tokenattributes;
+using Lucene.Net.Codecs;
+using Lucene.Net.Documents;
+using Lucene.Net.Index;
+using Lucene.Net.Store;
+using Lucene.Net.Support;
+using Lucene.Net.Util;
+using Lucene.Net.Util.Fst;
+using Directory = Lucene.Net.Store.Directory;
+using Version = Lucene.Net.Util.Version;
+
+namespace Lucene.Net.Search.Suggest.Analyzing
+{
+
+	/*
+	 * 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.
+	 */
+
+	// TODO
+	//   - test w/ syns
+	//   - add pruning of low-freq ngrams?   
+
+	/// <summary>
+	/// Builds an ngram model from the text sent to {@link
+	/// #build} and predicts based on the last grams-1 tokens in
+	/// the request sent to <seealso cref="#lookup"/>.  This tries to
+	/// handle the "long tail" of suggestions for when the
+	/// incoming query is a never before seen query string.
+	/// 
+	/// <para>Likely this suggester would only be used as a
+	/// fallback, when the primary suggester fails to find
+	/// any suggestions.
+	/// 
+	/// </para>
+	/// <para>Note that the weight for each suggestion is unused,
+	/// and the suggestions are the analyzed forms (so your
+	/// analysis process should normally be very "light").
+	/// 
+	/// </para>
+	/// <para>This uses the stupid backoff language model to smooth
+	/// scores across ngram models; see
+	/// "Large language models in machine translation",
+	/// http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.76.1126
+	/// for details.
+	/// 
+	/// </para>
+	/// <para> From <seealso cref="#lookup"/>, the key of each result is the
+	/// ngram token; the value is Long.MAX_VALUE * score (fixed
+	/// point, cast to long).  Divide by Long.MAX_VALUE to get
+	/// the score back, which ranges from 0.0 to 1.0.
+	/// 
+	/// onlyMorePopular is unused.
+	/// 
+	/// @lucene.experimental
+	/// </para>
+	/// </summary>
+	public class FreeTextSuggester : Lookup
+	{
+
+	  /// <summary>
+	  /// Codec name used in the header for the saved model. </summary>
+	  public const string CODEC_NAME = "freetextsuggest";
+
+	  /// <summary>
+	  /// Initial version of the the saved model file format. </summary>
+	  public const int VERSION_START = 0;
+
+	  /// <summary>
+	  /// Current version of the the saved model file format. </summary>
+	  public const int VERSION_CURRENT = VERSION_START;
+
+	  /// <summary>
+	  /// By default we use a bigram model. </summary>
+	  public const int DEFAULT_GRAMS = 2;
+
+	  // In general this could vary with gram, but the
+	  // original paper seems to use this constant:
+	  /// <summary>
+	  /// The constant used for backoff smoothing; during
+	  ///  lookup, this means that if a given trigram did not
+	  ///  occur, and we backoff to the bigram, the overall score
+	  ///  will be 0.4 times what the bigram model would have
+	  ///  assigned. 
+	  /// </summary>
+	  public const double ALPHA = 0.4;
+
+	  /// <summary>
+	  /// Holds 1gram, 2gram, 3gram models as a single FST. </summary>
+	  private FST<long?> fst;
+
+	  /// <summary>
+	  /// Analyzer that will be used for analyzing suggestions at
+	  /// index time.
+	  /// </summary>
+	  private readonly Analyzer indexAnalyzer;
+
+	  private long totTokens;
+
+	  /// <summary>
+	  /// Analyzer that will be used for analyzing suggestions at
+	  /// query time.
+	  /// </summary>
+	  private readonly Analyzer queryAnalyzer;
+
+	  // 2 = bigram, 3 = trigram
+	  private readonly int grams;
+
+	  private readonly sbyte separator;
+
+	  /// <summary>
+	  /// Number of entries the lookup was built with </summary>
+	  private long count = 0;
+
+	  /// <summary>
+	  /// The default character used to join multiple tokens
+	  ///  into a single ngram token.  The input tokens produced
+	  ///  by the analyzer must not contain this character. 
+	  /// </summary>
+	  public const sbyte DEFAULT_SEPARATOR = 0x1e;
+
+	  /// <summary>
+	  /// Instantiate, using the provided analyzer for both
+	  ///  indexing and lookup, using bigram model by default. 
+	  /// </summary>
+	  public FreeTextSuggester(Analyzer analyzer) : this(analyzer, analyzer, DEFAULT_GRAMS)
+	  {
+	  }
+
+	  /// <summary>
+	  /// Instantiate, using the provided indexing and lookup
+	  ///  analyzers, using bigram model by default. 
+	  /// </summary>
+	  public FreeTextSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) : this(indexAnalyzer, queryAnalyzer, DEFAULT_GRAMS)
+	  {
+	  }
+
+	  /// <summary>
+	  /// Instantiate, using the provided indexing and lookup
+	  ///  analyzers, with the specified model (2
+	  ///  = bigram, 3 = trigram, etc.). 
+	  /// </summary>
+	  public FreeTextSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer, int grams) : this(indexAnalyzer, queryAnalyzer, grams, DEFAULT_SEPARATOR)
+	  {
+	  }
+
+	  /// <summary>
+	  /// Instantiate, using the provided indexing and lookup
+	  ///  analyzers, and specified model (2 = bigram, 3 =
+	  ///  trigram ,etc.).  The separator is passed to {@link
+	  ///  ShingleFilter#setTokenSeparator} to join multiple
+	  ///  tokens into a single ngram token; it must be an ascii
+	  ///  (7-bit-clean) byte.  No input tokens should have this
+	  ///  byte, otherwise {@code IllegalArgumentException} is
+	  ///  thrown. 
+	  /// </summary>
+	  public FreeTextSuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer, int grams, sbyte separator)
+	  {
+		this.grams = grams;
+		this.indexAnalyzer = AddShingles(indexAnalyzer);
+		this.queryAnalyzer = AddShingles(queryAnalyzer);
+		if (grams < 1)
+		{
+		  throw new System.ArgumentException("grams must be >= 1");
+		}
+		if ((separator & 0x80) != 0)
+		{
+		  throw new System.ArgumentException("separator must be simple ascii character");
+		}
+		this.separator = separator;
+	  }
+
+	  /// <summary>
+	  /// Returns byte size of the underlying FST. </summary>
+	  public override long SizeInBytes()
+	  {
+		if (fst == null)
+		{
+		  return 0;
+		}
+		return fst.SizeInBytes();
+	  }
+
+	  private class AnalyzingComparator : IComparer<BytesRef>
+	  {
+
+		internal readonly ByteArrayDataInput readerA = new ByteArrayDataInput();
+		internal readonly ByteArrayDataInput readerB = new ByteArrayDataInput();
+		internal readonly BytesRef scratchA = new BytesRef();
+		internal readonly BytesRef scratchB = new BytesRef();
+
+		public virtual int Compare(BytesRef a, BytesRef b)
+		{
+		  readerA.Reset(a.Bytes, a.Offset, a.Length);
+		  readerB.Reset(b.Bytes, b.Offset, b.Length);
+
+		  // By token:
+		  scratchA.Length = readerA.ReadShort();
+		  scratchA.Bytes = a.Bytes;
+		  scratchA.Offset = readerA.Position;
+
+		  scratchB.Bytes = b.Bytes;
+		  scratchB.Length = readerB.ReadShort();
+		  scratchB.Offset = readerB.Position;
+
+		  int cmp = scratchA.CompareTo(scratchB);
+		  if (cmp != 0)
+		  {
+			return cmp;
+		  }
+		  readerA.SkipBytes(scratchA.Length);
+		  readerB.SkipBytes(scratchB.Length);
+
+		  // By length (smaller surface forms sorted first):
+		  cmp = a.Length - b.Length;
+		  if (cmp != 0)
+		  {
+			return cmp;
+		  }
+
+		  // By surface form:
+		  scratchA.Offset = readerA.Position;
+		  scratchA.Length = a.Length - scratchA.Offset;
+		  scratchB.Offset = readerB.Position;
+		  scratchB.Length = b.Length - scratchB.Offset;
+
+		  return scratchA.CompareTo(scratchB);
+		}
+	  }
+
+	  private Analyzer AddShingles(Analyzer other)
+	  {
+		if (grams == 1)
+		{
+		  return other;
+		}
+		else
+		{
+		  // TODO: use ShingleAnalyzerWrapper?
+		  // Tack on ShingleFilter to the end, to generate token ngrams:
+		  return new AnalyzerWrapperAnonymousInnerClassHelper(this, other.ReuseStrategy, other);
+		}
+	  }
+
+	  private class AnalyzerWrapperAnonymousInnerClassHelper : AnalyzerWrapper
+	  {
+		  private readonly FreeTextSuggester outerInstance;
+		  private readonly Analyzer other;
+
+		  public AnalyzerWrapperAnonymousInnerClassHelper(FreeTextSuggester outerInstance, UnknownType getReuseStrategy, Analyzer other) : base(getReuseStrategy)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.other = other;
+		  }
+
+		  protected override Analyzer GetWrappedAnalyzer(string fieldName)
+		  {
+			return other;
+		  }
+
+		  protected override TokenStreamComponents WrapComponents(string fieldName, TokenStreamComponents components)
+		  {
+			ShingleFilter shingles = new ShingleFilter(components.TokenStream, 2, outerInstance.grams);
+			shingles.TokenSeparator = char.ToString((char) outerInstance.separator);
+			return new TokenStreamComponents(components.Tokenizer, shingles);
+		  }
+	  }
+
+	  public override void Build(InputIterator iterator)
+	  {
+		Build(iterator, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB);
+	  }
+
+	  /// <summary>
+	  /// Build the suggest index, using up to the specified
+	  ///  amount of temporary RAM while building.  Note that
+	  ///  the weights for the suggestions are ignored. 
+	  /// </summary>
+	  public virtual void Build(InputIterator iterator, double ramBufferSizeMB)
+	  {
+		if (iterator.HasPayloads())
+		{
+		  throw new System.ArgumentException("this suggester doesn't support payloads");
+		}
+		if (iterator.HasContexts())
+		{
+		  throw new System.ArgumentException("this suggester doesn't support contexts");
+		}
+
+		string prefix = this.GetType().Name;
+		var directory = OfflineSorter.DefaultTempDir();
+		// TODO: messy ... java7 has Files.createTempDirectory
+		// ... but 4.x is java6:
+		File tempIndexPath = null;
+		Random random = new Random();
+		while (true)
+		{
+		  tempIndexPath = new File(directory, prefix + ".index." + random.Next(int.MaxValue));
+		  if (tempIndexPath.mkdir())
+		  {
+			break;
+		  }
+		}
+
+		Directory dir = FSDirectory.Open(tempIndexPath);
+
+		IndexWriterConfig iwc = new IndexWriterConfig(Version.LUCENE_CURRENT, indexAnalyzer);
+		iwc.OpenMode = IndexWriterConfig.OpenMode.CREATE;
+		iwc.RAMBufferSizeMB = ramBufferSizeMB;
+		IndexWriter writer = new IndexWriter(dir, iwc);
+
+		FieldType ft = new FieldType(TextField.TYPE_NOT_STORED);
+		// TODO: if only we had IndexOptions.TERMS_ONLY...
+		ft.IndexOptions = FieldInfo.IndexOptions.DOCS_AND_FREQS;
+		ft.OmitNorms = true;
+		ft.Freeze();
+
+		Document doc = new Document();
+		Field field = new Field("body", "", ft);
+		doc.Add(field);
+
+		totTokens = 0;
+		IndexReader reader = null;
+
+		bool success = false;
+		count = 0;
+		try
+		{
+		  while (true)
+		  {
+			BytesRef surfaceForm = iterator.Next();
+			if (surfaceForm == null)
+			{
+			  break;
+			}
+			field.StringValue = surfaceForm.Utf8ToString();
+			writer.AddDocument(doc);
+			count++;
+		  }
+		  reader = DirectoryReader.Open(writer, false);
+
+		  Terms terms = MultiFields.GetTerms(reader, "body");
+		  if (terms == null)
+		  {
+			throw new System.ArgumentException("need at least one suggestion");
+		  }
+
+		  // Move all ngrams into an FST:
+		  TermsEnum termsEnum = terms.Iterator(null);
+
+		  Outputs<long?> outputs = PositiveIntOutputs.Singleton;
+		  Builder<long?> builder = new Builder<long?>(FST.INPUT_TYPE.BYTE1, outputs);
+
+		  IntsRef scratchInts = new IntsRef();
+		  while (true)
+		  {
+			BytesRef term = termsEnum.next();
+			if (term == null)
+			{
+			  break;
+			}
+			int ngramCount = countGrams(term);
+			if (ngramCount > grams)
+			{
+			  throw new System.ArgumentException("tokens must not contain separator byte; got token=" + term + " but gramCount=" + ngramCount + ", which is greater than expected max ngram size=" + grams);
+			}
+			if (ngramCount == 1)
+			{
+			  totTokens += termsEnum.TotalTermFreq();
+			}
+
+			builder.Add(Util.ToIntsRef(term, scratchInts), encodeWeight(termsEnum.TotalTermFreq()));
+		  }
+
+		  fst = builder.Finish();
+		  if (fst == null)
+		  {
+			throw new System.ArgumentException("need at least one suggestion");
+		  }
+		  //System.out.println("FST: " + fst.getNodeCount() + " nodes");
+
+		  /*
+		  PrintWriter pw = new PrintWriter("/x/tmp/out.dot");
+		  Util.toDot(fst, pw, true, true);
+		  pw.close();
+		  */
+
+		  success = true;
+		}
+		finally
+		{
+		  try
+		  {
+			if (success)
+			{
+			  IOUtils.Close(writer, reader);
+			}
+			else
+			{
+			  IOUtils.CloseWhileHandlingException(writer, reader);
+			}
+		  }
+		  finally
+		  {
+			foreach (string file in dir.ListAll())
+			{
+			  File path = new File(tempIndexPath, file);
+			  if (path.Delete() == false)
+			  {
+				throw new InvalidOperationException("failed to remove " + path);
+			  }
+			}
+
+			if (tempIndexPath.Delete() == false)
+			{
+			  throw new InvalidOperationException("failed to remove " + tempIndexPath);
+			}
+
+			dir.Dispose();
+		  }
+		}
+	  }
+
+	  public override bool Store(DataOutput output)
+	  {
+		CodecUtil.WriteHeader(output, CODEC_NAME, VERSION_CURRENT);
+		output.WriteVLong(count);
+		output.WriteByte(separator);
+		output.WriteVInt(grams);
+		output.WriteVLong(totTokens);
+		fst.Save(output);
+		return true;
+	  }
+
+	  public override bool Load(DataInput input)
+	  {
+		CodecUtil.CheckHeader(input, CODEC_NAME, VERSION_START, VERSION_START);
+		count = input.ReadVLong();
+		sbyte separatorOrig = input.ReadByte();
+		if (separatorOrig != separator)
+		{
+		  throw new InvalidOperationException("separator=" + separator + " is incorrect: original model was built with separator=" + separatorOrig);
+		}
+		int gramsOrig = input.ReadVInt();
+		if (gramsOrig != grams)
+		{
+		  throw new InvalidOperationException("grams=" + grams + " is incorrect: original model was built with grams=" + gramsOrig);
+		}
+		totTokens = input.ReadVLong();
+
+		fst = new FST<>(input, PositiveIntOutputs.Singleton);
+
+		return true;
+	  }
+
+	  public override IList<LookupResult> Lookup(string key, bool onlyMorePopular, int num) // ignored
+	  {
+		return Lookup(key, null, onlyMorePopular, num);
+	  }
+
+	  /// <summary>
+	  /// Lookup, without any context. </summary>
+	  public virtual IList<LookupResult> Lookup(string key, int num)
+	  {
+		return Lookup(key, null, true, num);
+	  }
+
+	  public override IList<LookupResult> Lookup(string key, HashSet<BytesRef> contexts, bool onlyMorePopular, int num) // ignored
+	  {
+		try
+		{
+		  return Lookup(key, contexts, num);
+		}
+		catch (IOException ioe)
+		{
+		  // bogus:
+		  throw new Exception(ioe);
+		}
+	  }
+
+	  public override long Count
+	  {
+		  get
+		  {
+			return count;
+		  }
+	  }
+
+	  private int CountGrams(BytesRef token)
+	  {
+		int count = 1;
+		for (int i = 0;i < token.Length;i++)
+		{
+		  if (token.Bytes[token.Offset + i] == separator)
+		  {
+			count++;
+		  }
+		}
+
+		return count;
+	  }
+
+	  /// <summary>
+	  /// Retrieve suggestions.
+	  /// </summary>
+	  public virtual IList<LookupResult> Lookup(string key, HashSet<BytesRef> contexts, int num)
+	  {
+		if (contexts != null)
+		{
+		  throw new System.ArgumentException("this suggester doesn't support contexts");
+		}
+
+		TokenStream ts = queryAnalyzer.TokenStream("", key.ToString());
+		try
+		{
+		  TermToBytesRefAttribute termBytesAtt = ts.AddAttribute<TermToBytesRefAttribute>();
+		  OffsetAttribute offsetAtt = ts.AddAttribute<OffsetAttribute>();
+		  PositionLengthAttribute posLenAtt = ts.AddAttribute<PositionLengthAttribute>();
+		  PositionIncrementAttribute posIncAtt = ts.AddAttribute<PositionIncrementAttribute>();
+		  ts.Reset();
+
+		  var lastTokens = new BytesRef[grams];
+		  //System.out.println("lookup: key='" + key + "'");
+
+		  // Run full analysis, but save only the
+		  // last 1gram, last 2gram, etc.:
+		  BytesRef tokenBytes = termBytesAtt.BytesRef;
+		  int maxEndOffset = -1;
+		  bool sawRealToken = false;
+		  while (ts.IncrementToken())
+		  {
+			termBytesAtt.FillBytesRef();
+			sawRealToken |= tokenBytes.Length > 0;
+			// TODO: this is somewhat iffy; today, ShingleFilter
+			// sets posLen to the gram count; maybe we should make
+			// a separate dedicated att for this?
+			int gramCount = posLenAtt.PositionLength;
+
+			Debug.Assert(gramCount <= grams);
+
+			// Safety: make sure the recalculated count "agrees":
+			if (CountGrams(tokenBytes) != gramCount)
+			{
+			  throw new System.ArgumentException("tokens must not contain separator byte; got token=" + tokenBytes + " but gramCount=" + gramCount + " does not match recalculated count=" + countGrams(tokenBytes));
+			}
+			maxEndOffset = Math.Max(maxEndOffset, offsetAtt.EndOffset());
+			lastTokens[gramCount - 1] = BytesRef.DeepCopyOf(tokenBytes);
+		  }
+		  ts.End();
+
+		  if (!sawRealToken)
+		  {
+			throw new System.ArgumentException("no tokens produced by analyzer, or the only tokens were empty strings");
+		  }
+
+		  // Carefully fill last tokens with _ tokens;
+		  // ShingleFilter appraently won't emit "only hole"
+		  // tokens:
+		  int endPosInc = posIncAtt.PositionIncrement;
+
+		  // Note this will also be true if input is the empty
+		  // string (in which case we saw no tokens and
+		  // maxEndOffset is still -1), which in fact works out OK
+		  // because we fill the unigram with an empty BytesRef
+		  // below:
+		  bool lastTokenEnded = offsetAtt.EndOffset() > maxEndOffset || endPosInc > 0;
+		  //System.out.println("maxEndOffset=" + maxEndOffset + " vs " + offsetAtt.endOffset());
+
+		  if (lastTokenEnded)
+		  {
+			//System.out.println("  lastTokenEnded");
+			// If user hit space after the last token, then
+			// "upgrade" all tokens.  This way "foo " will suggest
+			// all bigrams starting w/ foo, and not any unigrams
+			// starting with "foo":
+			for (int i = grams - 1;i > 0;i--)
+			{
+			  BytesRef token = lastTokens[i - 1];
+			  if (token == null)
+			  {
+				continue;
+			  }
+			  token.Grow(token.Length + 1);
+			  token.Bytes[token.Length] = separator;
+			  token.Length++;
+			  lastTokens[i] = token;
+			}
+			lastTokens[0] = new BytesRef();
+		  }
+
+		  FST.Arc<long?> arc = new FST.Arc<long?>();
+
+		  FST.BytesReader bytesReader = fst.BytesReader;
+
+		  // Try highest order models first, and if they return
+		  // results, return that; else, fallback:
+		  double backoff = 1.0;
+
+		  IList<LookupResult> results = new List<LookupResult>(num);
+
+		  // We only add a given suffix once, from the highest
+		  // order model that saw it; for subsequent lower order
+		  // models we skip it:
+		  var seen = new HashSet<BytesRef>();
+
+		  for (int gram = grams - 1;gram >= 0;gram--)
+		  {
+			BytesRef token = lastTokens[gram];
+			// Don't make unigram predictions from empty string:
+			if (token == null || (token.Length == 0 && key.Length > 0))
+			{
+			  // Input didn't have enough tokens:
+			  //System.out.println("  gram=" + gram + ": skip: not enough input");
+			  continue;
+			}
+
+			if (endPosInc > 0 && gram <= endPosInc)
+			{
+			  // Skip hole-only predictions; in theory we
+			  // shouldn't have to do this, but we'd need to fix
+			  // ShingleFilter to produce only-hole tokens:
+			  //System.out.println("  break: only holes now");
+			  break;
+			}
+
+			//System.out.println("try " + (gram+1) + " gram token=" + token.utf8ToString());
+
+			// TODO: we could add fuzziness here
+			// match the prefix portion exactly
+			//Pair<Long,BytesRef> prefixOutput = null;
+			long? prefixOutput = null;
+			try
+			{
+			  prefixOutput = LookupPrefix(fst, bytesReader, token, arc);
+			}
+			catch (IOException bogus)
+			{
+			  throw new Exception(bogus);
+			}
+			//System.out.println("  prefixOutput=" + prefixOutput);
+
+			if (prefixOutput == null)
+			{
+			  // This model never saw this prefix, e.g. the
+			  // trigram model never saw context "purple mushroom"
+			  backoff *= ALPHA;
+			  continue;
+			}
+
+			// TODO: we could do this division at build time, and
+			// bake it into the FST?
+
+			// Denominator for computing scores from current
+			// model's predictions:
+			long contextCount = totTokens;
+
+			BytesRef lastTokenFragment = null;
+
+			for (int i = token.Length - 1;i >= 0;i--)
+			{
+			  if (token.Bytes[token.Offset + i] == separator)
+			  {
+				BytesRef context = new BytesRef(token.Bytes, token.Offset, i);
+				long? output = Util.Get(fst, Util.ToIntsRef(context, new IntsRef()));
+				Debug.Assert(output != null);
+				contextCount = DecodeWeight(output);
+				lastTokenFragment = new BytesRef(token.Bytes, token.Offset + i + 1, token.Length - i - 1);
+				break;
+			  }
+			}
+
+			BytesRef finalLastToken;
+
+			if (lastTokenFragment == null)
+			{
+			  finalLastToken = BytesRef.DeepCopyOf(token);
+			}
+			else
+			{
+			  finalLastToken = BytesRef.DeepCopyOf(lastTokenFragment);
+			}
+			Debug.Assert(finalLastToken.Offset == 0);
+
+			CharsRef spare = new CharsRef();
+
+			// complete top-N
+			Util.TopResults<long?> completions = null;
+			try
+			{
+
+			  // Because we store multiple models in one FST
+			  // (1gram, 2gram, 3gram), we must restrict the
+			  // search so that it only considers the current
+			  // model.  For highest order model, this is not
+			  // necessary since all completions in the FST
+			  // must be from this model, but for lower order
+			  // models we have to filter out the higher order
+			  // ones:
+
+			  // Must do num+seen.size() for queue depth because we may
+			  // reject up to seen.size() paths in acceptResult():
+			  Util.TopNSearcher<long?> searcher = new TopNSearcherAnonymousInnerClassHelper(this, fst, num, num + seen.Count, weightComparator, seen, finalLastToken);
+
+			  // since this search is initialized with a single start node 
+			  // it is okay to start with an empty input path here
+			  searcher.AddStartPaths(arc, prefixOutput, true, new IntsRef());
+
+			  completions = searcher.Search();
+			  Debug.Assert(completions.IsComplete);
+			}
+			catch (IOException bogus)
+			{
+			  throw new Exception(bogus);
+			}
+
+			int prefixLength = token.Length;
+
+			BytesRef suffix = new BytesRef(8);
+			//System.out.println("    " + completions.length + " completions");
+
+			  foreach (Util.Result<long?> completion in completions)
+			  {
+				token.Length = prefixLength;
+				// append suffix
+				Util.ToBytesRef(completion.Input, suffix);
+				token.Append(suffix);
+
+				//System.out.println("    completion " + token.utf8ToString());
+
+				// Skip this path if a higher-order model already
+				// saw/predicted its last token:
+				BytesRef lastToken = token;
+				for (int i = token.Length - 1;i >= 0;i--)
+				{
+				  if (token.Bytes[token.Offset + i] == separator)
+				  {
+					Debug.Assert(token.Length - i - 1 > 0);
+					lastToken = new BytesRef(token.Bytes, token.Offset + i + 1, token.Length - i - 1);
+					break;
+				  }
+				}
+				if (seen.Contains(lastToken))
+				{
+				  //System.out.println("      skip dup " + lastToken.utf8ToString());
+				  goto nextCompletionContinue;
+				}
+				seen.Add(BytesRef.DeepCopyOf(lastToken));
+				spare.Grow(token.Length);
+				UnicodeUtil.UTF8toUTF16(token, spare);
+				LookupResult result = new LookupResult(spare.ToString(), (long)(long.MaxValue * backoff * ((double) decodeWeight(completion.Output)) / contextCount));
+				results.Add(result);
+				Debug.Assert(results.Count == seen.Count);
+				//System.out.println("  add result=" + result);
+				nextCompletionContinue:;
+			  }
+			nextCompletionBreak:
+			backoff *= ALPHA;
+		  }
+
+		  results.Sort(new ComparatorAnonymousInnerClassHelper(this));
+
+		  if (results.Count > num)
+		  {
+			results.SubList(num, results.Count).Clear();
+		  }
+
+		  return results;
+		}
+		finally
+		{
+		  IOUtils.CloseWhileHandlingException(ts);
+		}
+	  }
+
+	  private class TopNSearcherAnonymousInnerClassHelper : Util.TopNSearcher<long?>
+	  {
+		  private readonly FreeTextSuggester outerInstance;
+
+		  private HashSet<BytesRef> seen;
+		  private BytesRef finalLastToken;
+
+		  public TopNSearcherAnonymousInnerClassHelper<T1>(FreeTextSuggester outerInstance, FST<T1> org.apache.lucene.search.suggest.fst, int num, UnknownType size, UnknownType weightComparator, HashSet<BytesRef> seen, BytesRef finalLastToken) : base(org.apache.lucene.search.suggest.fst, num, size, weightComparator)
+		  {
+			  this.outerInstance = outerInstance;
+			  this.seen = seen;
+			  this.finalLastToken = finalLastToken;
+			  scratchBytes = new BytesRef();
+		  }
+
+
+		  internal BytesRef scratchBytes;
+
+		  protected internal override void addIfCompetitive(Util.FSTPath<long?> path)
+		  {
+			if (path.Arc.label != outerInstance.separator)
+			{
+			  //System.out.println("    keep path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
+			  base.AddIfCompetitive(path);
+			}
+			else
+			{
+			  //System.out.println("    prevent path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
+			}
+		  }
+
+		  protected internal override bool AcceptResult(IntsRef input, long? output)
+		  {
+			Util.ToBytesRef(input, scratchBytes);
+			finalLastToken.Grow(finalLastToken.length + scratchBytes.length);
+			int lenSav = finalLastToken.length;
+			finalLastToken.append(scratchBytes);
+			//System.out.println("    accept? input='" + scratchBytes.utf8ToString() + "'; lastToken='" + finalLastToken.utf8ToString() + "'; return " + (seen.contains(finalLastToken) == false));
+			bool ret = seen.Contains(finalLastToken) == false;
+
+			finalLastToken.length = lenSav;
+			return ret;
+		  }
+	  }
+
+	  private class ComparatorAnonymousInnerClassHelper : IComparer<Lookup.LookupResult>
+	  {
+		  private readonly FreeTextSuggester outerInstance;
+
+		  public ComparatorAnonymousInnerClassHelper(FreeTextSuggester outerInstance)
+		  {
+			  this.outerInstance = outerInstance;
+		  }
+
+		  public virtual int Compare(LookupResult a, LookupResult b)
+		  {
+			if (a.value > b.value)
+			{
+			  return -1;
+			}
+			else if (a.value < b.value)
+			{
+			  return 1;
+			}
+			else
+			{
+			  // Tie break by UTF16 sort order:
+			  return ((string) a.key).CompareTo((string) b.key);
+			}
+		  }
+	  }
+
+	  /// <summary>
+	  /// weight -> cost </summary>
+	  private long EncodeWeight(long ngramCount)
+	  {
+		return long.MaxValue - ngramCount;
+	  }
+
+	  /// <summary>
+	  /// cost -> weight </summary>
+	  //private long decodeWeight(Pair<Long,BytesRef> output) {
+	  private long DecodeWeight(long? output)
+	  {
+		Debug.Assert(output != null);
+		return (int)(long.MaxValue - output);
+	  }
+
+	  // NOTE: copied from WFSTCompletionLookup & tweaked
+	  private long? LookupPrefix(FST<long?> fst, FST.BytesReader bytesReader, BytesRef scratch, FST.Arc<long?> arc) //Bogus
+	  {
+
+		long? output = fst.outputs.NoOutput;
+
+		fst.GetFirstArc(arc);
+
+		sbyte[] bytes = scratch.Bytes;
+		int pos = scratch.Offset;
+		int end = pos + scratch.Length;
+		while (pos < end)
+		{
+		  if (fst.FindTargetArc(bytes[pos++] & 0xff, arc, arc, bytesReader) == null)
+		  {
+			return null;
+		  }
+		  else
+		  {
+			output = fst.outputs.add(output, arc.output);
+		  }
+		}
+
+		return output;
+	  }
+
+	  internal static readonly IComparer<long?> weightComparator = new ComparatorAnonymousInnerClassHelper2();
+
+	  private class ComparatorAnonymousInnerClassHelper2 : IComparer<long?>
+	  {
+		  public ComparatorAnonymousInnerClassHelper2()
+		  {
+		  }
+
+		  public virtual int Compare(long? left, long? right)
+		  {
+			return left.CompareTo(right);
+		  }
+	  }
+
+	  /// <summary>
+	  /// Returns the weight associated with an input string,
+	  /// or null if it does not exist.
+	  /// </summary>
+	  public virtual object Get(string key)
+	  {
+		throw new System.NotSupportedException();
+	  }
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/Analyzing/FuzzySuggester.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/Analyzing/FuzzySuggester.cs b/src/Lucene.Net.Suggest/Suggest/Analyzing/FuzzySuggester.cs
new file mode 100644
index 0000000..df65851
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/Analyzing/FuzzySuggester.cs
@@ -0,0 +1,271 @@
+using System;
+using System.Collections.Generic;
+using Lucene.Net.Analysis;
+using Lucene.Net.Support;
+using Lucene.Net.Util;
+using Lucene.Net.Util.Automaton;
+
+namespace Lucene.Net.Search.Suggest.Analyzing
+{
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// Implements a fuzzy <seealso cref="AnalyzingSuggester"/>. The similarity measurement is
+    /// based on the Damerau-Levenshtein (optimal string alignment) algorithm, though
+    /// you can explicitly choose classic Levenshtein by passing <code>false</code>
+    /// for the <code>transpositions</code> parameter.
+    /// <para>
+    /// At most, this query will match terms up to
+    /// {@value org.apache.lucene.util.automaton.LevenshteinAutomata#MAXIMUM_SUPPORTED_DISTANCE}
+    /// edits. Higher distances are not supported.  Note that the
+    /// fuzzy distance is measured in "byte space" on the bytes
+    /// returned by the <seealso cref="TokenStream"/>'s {@link
+    /// TermToBytesRefAttribute}, usually UTF8.  By default
+    /// the analyzed bytes must be at least 3 {@link
+    /// #DEFAULT_MIN_FUZZY_LENGTH} bytes before any edits are
+    /// considered.  Furthermore, the first 1 {@link
+    /// #DEFAULT_NON_FUZZY_PREFIX} byte is not allowed to be
+    /// edited.  We allow up to 1 (@link
+    /// #DEFAULT_MAX_EDITS} edit.
+    /// If <seealso cref="#unicodeAware"/> parameter in the constructor is set to true, maxEdits,
+    /// minFuzzyLength, transpositions and nonFuzzyPrefix are measured in Unicode code 
+    /// points (actual letters) instead of bytes. 
+    /// 
+    /// </para>
+    /// <para>
+    /// NOTE: This suggester does not boost suggestions that
+    /// required no edits over suggestions that did require
+    /// edits.  This is a known limitation.
+    /// 
+    /// </para>
+    /// <para>
+    /// Note: complex query analyzers can have a significant impact on the lookup
+    /// performance. It's recommended to not use analyzers that drop or inject terms
+    /// like synonyms to keep the complexity of the prefix intersection low for good
+    /// lookup performance. At index time, complex analyzers can safely be used.
+    /// </para>
+    /// 
+    /// @lucene.experimental
+    /// </summary>
+    public sealed class FuzzySuggester : AnalyzingSuggester
+    {
+        private readonly int maxEdits;
+        private readonly bool transpositions;
+        private readonly int nonFuzzyPrefix;
+        private readonly int minFuzzyLength;
+        private readonly bool unicodeAware;
+
+        /// <summary>
+        /// Measure maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix 
+        ///  parameters in Unicode code points (actual letters)
+        ///  instead of bytes. 
+        /// </summary>
+        public const bool DEFAULT_UNICODE_AWARE = false;
+
+        /// <summary>
+        /// The default minimum length of the key passed to {@link
+        /// #lookup} before any edits are allowed.
+        /// </summary>
+        public const int DEFAULT_MIN_FUZZY_LENGTH = 3;
+
+        /// <summary>
+        /// The default prefix length where edits are not allowed.
+        /// </summary>
+        public const int DEFAULT_NON_FUZZY_PREFIX = 1;
+
+        /// <summary>
+        /// The default maximum number of edits for fuzzy
+        /// suggestions.
+        /// </summary>
+        public const int DEFAULT_MAX_EDITS = 1;
+
+        /// <summary>
+        /// The default transposition value passed to <seealso cref="LevenshteinAutomata"/>
+        /// </summary>
+        public const bool DEFAULT_TRANSPOSITIONS = true;
+
+        /// <summary>
+        /// Creates a <seealso cref="FuzzySuggester"/> instance initialized with default values.
+        /// </summary>
+        /// <param name="analyzer"> the analyzer used for this suggester </param>
+        public FuzzySuggester(Analyzer analyzer)
+            : this(analyzer, analyzer)
+        {
+        }
+
+        /// <summary>
+        /// Creates a <seealso cref="FuzzySuggester"/> instance with an index & a query analyzer initialized with default values.
+        /// </summary>
+        /// <param name="indexAnalyzer">
+        ///           Analyzer that will be used for analyzing suggestions while building the index. </param>
+        /// <param name="queryAnalyzer">
+        ///           Analyzer that will be used for analyzing query text during lookup </param>
+        public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer)
+            : this(indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, true, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS, DEFAULT_NON_FUZZY_PREFIX, DEFAULT_MIN_FUZZY_LENGTH, DEFAULT_UNICODE_AWARE)
+        {
+        }
+
+        /// <summary>
+        /// Creates a <seealso cref="FuzzySuggester"/> instance.
+        /// </summary>
+        /// <param name="indexAnalyzer"> Analyzer that will be used for
+        ///        analyzing suggestions while building the index. </param>
+        /// <param name="queryAnalyzer"> Analyzer that will be used for
+        ///        analyzing query text during lookup </param>
+        /// <param name="options"> see <seealso cref="#EXACT_FIRST"/>, <seealso cref="#PRESERVE_SEP"/> </param>
+        /// <param name="maxSurfaceFormsPerAnalyzedForm"> Maximum number of
+        ///        surface forms to keep for a single analyzed form.
+        ///        When there are too many surface forms we discard the
+        ///        lowest weighted ones. </param>
+        /// <param name="maxGraphExpansions"> Maximum number of graph paths
+        ///        to expand from the analyzed form.  Set this to -1 for
+        ///        no limit. </param>
+        /// <param name="preservePositionIncrements"> Whether position holes should appear in the automaton </param>
+        /// <param name="maxEdits"> must be >= 0 and <= <seealso cref="LevenshteinAutomata#MAXIMUM_SUPPORTED_DISTANCE"/> . </param>
+        /// <param name="transpositions"> <code>true</code> if transpositions should be treated as a primitive 
+        ///        edit operation. If this is false, comparisons will implement the classic
+        ///        Levenshtein algorithm. </param>
+        /// <param name="nonFuzzyPrefix"> length of common (non-fuzzy) prefix (see default <seealso cref="#DEFAULT_NON_FUZZY_PREFIX"/> </param>
+        /// <param name="minFuzzyLength"> minimum length of lookup key before any edits are allowed (see default <seealso cref="#DEFAULT_MIN_FUZZY_LENGTH"/>) </param>
+        /// <param name="unicodeAware"> operate Unicode code points instead of bytes. </param>
+        public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer, int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions, bool preservePositionIncrements, int maxEdits, bool transpositions, int nonFuzzyPrefix, int minFuzzyLength, bool unicodeAware)
+            : base(indexAnalyzer, queryAnalyzer, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions, preservePositionIncrements)
+        {
+            if (maxEdits < 0 || maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE)
+            {
+                throw new System.ArgumentException("maxEdits must be between 0 and " + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE);
+            }
+            if (nonFuzzyPrefix < 0)
+            {
+                throw new System.ArgumentException("nonFuzzyPrefix must not be >= 0 (got " + nonFuzzyPrefix + ")");
+            }
+            if (minFuzzyLength < 0)
+            {
+                throw new System.ArgumentException("minFuzzyLength must not be >= 0 (got " + minFuzzyLength + ")");
+            }
+
+            this.maxEdits = maxEdits;
+            this.transpositions = transpositions;
+            this.nonFuzzyPrefix = nonFuzzyPrefix;
+            this.minFuzzyLength = minFuzzyLength;
+            this.unicodeAware = unicodeAware;
+        }
+
+        protected internal override IList<FSTUtil.Path<Pair<long?, BytesRef>>> GetFullPrefixPaths(IList<FSTUtil.Path<Pair<long?, BytesRef>>> prefixPaths, Automaton lookupAutomaton, FST<Pair<long?, BytesRef>> fst)
+        {
+
+            // TODO: right now there's no penalty for fuzzy/edits,
+            // ie a completion whose prefix matched exactly what the
+            // user typed gets no boost over completions that
+            // required an edit, which get no boost over completions
+            // requiring two edits.  I suspect a multiplicative
+            // factor is appropriate (eg, say a fuzzy match must be at
+            // least 2X better weight than the non-fuzzy match to
+            // "compete") ... in which case I think the wFST needs
+            // to be log weights or something ...
+
+            Automaton levA = convertAutomaton(ToLevenshteinAutomata(lookupAutomaton));
+            /*
+              Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), StandardCharsets.UTF_8);
+              w.write(levA.toDot());
+              w.close();
+              System.out.println("Wrote LevA to out.dot");
+            */
+            return FSTUtil.IntersectPrefixPaths(levA, fst);
+        }
+
+        protected internal override Automaton ConvertAutomaton(Automaton a)
+        {
+            if (unicodeAware)
+            {
+                Automaton utf8automaton = (new UTF32ToUTF8()).Convert(a);
+                BasicOperations.Determinize(utf8automaton);
+                return utf8automaton;
+            }
+            else
+            {
+                return a;
+            }
+        }
+
+        internal override TokenStreamToAutomaton TokenStreamToAutomaton
+        {
+            get
+            {
+                var tsta = base.TokenStreamToAutomaton;
+                tsta.UnicodeArcs = unicodeAware;
+                return tsta;
+            }
+        }
+
+        internal Automaton ToLevenshteinAutomata(Automaton automaton)
+        {
+            var @ref = SpecialOperations.GetFiniteStrings(automaton, -1);
+            Automaton[] subs = new Automaton[@ref.Count];
+            int upto = 0;
+            foreach (IntsRef path in @ref)
+            {
+                if (path.Length <= nonFuzzyPrefix || path.Length < minFuzzyLength)
+                {
+                    subs[upto] = BasicAutomata.MakeString(path.Ints, path.Offset, path.Length);
+                    upto++;
+                }
+                else
+                {
+                    Automaton prefix = BasicAutomata.MakeString(path.Ints, path.Offset, nonFuzzyPrefix);
+                    int[] ints = new int[path.Length - nonFuzzyPrefix];
+                    Array.Copy(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 ? char.MAX_CODE_POINT : 255, transpositions);
+                    Automaton levAutomaton = lev.ToAutomaton(maxEdits);
+                    Automaton combined = BasicOperations.Concatenate(Arrays.AsList(prefix, levAutomaton));
+                    combined.Deterministic = true; // its like the special case in concatenate itself, except we cloneExpanded already
+                    subs[upto] = combined;
+                    upto++;
+                }
+            }
+
+            if (subs.Length == 0)
+            {
+                // automaton is empty, there is no accepted paths through it
+                return BasicAutomata.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 = BasicOperations.Union(Arrays.AsList(subs));
+                // TODO: we could call toLevenshteinAutomata() before det? 
+                // this only happens if you have multiple paths anyway (e.g. synonyms)
+                BasicOperations.Determinize(a);
+
+                return a;
+            }
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/Analyzing/SuggestStopFilter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/Analyzing/SuggestStopFilter.cs b/src/Lucene.Net.Suggest/Suggest/Analyzing/SuggestStopFilter.cs
new file mode 100644
index 0000000..3f2f72c
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/Analyzing/SuggestStopFilter.cs
@@ -0,0 +1,138 @@
+using System.Diagnostics;
+using Lucene.Net.Analysis;
+using Lucene.Net.Analysis.Tokenattributes;
+
+namespace Lucene.Net.Search.Suggest.Analyzing
+{
+
+    /*
+     * 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.
+     */
+    /// <summary>
+    /// Like <seealso cref="StopFilter"/> except it will not remove the
+    ///  last token if that token was not followed by some token
+    ///  separator.  For example, a query 'find the' would
+    ///  preserve the 'the' since it was not followed by a space or
+    ///  punctuation or something, and mark it KEYWORD so future
+    ///  stemmers won't touch it either while a query like "find
+    ///  the popsicle' would remove 'the' as a stopword.
+    /// 
+    ///  <para>Normally you'd use the ordinary <seealso cref="StopFilter"/>
+    ///  in your indexAnalyzer and then this class in your
+    ///  queryAnalyzer, when using one of the analyzing suggesters. 
+    /// </para>
+    /// </summary>
+
+    public sealed class SuggestStopFilter : TokenFilter
+    {
+
+        private readonly CharTermAttribute termAtt = addAttribute(typeof(CharTermAttribute));
+        private readonly PositionIncrementAttribute posIncAtt = addAttribute(typeof(PositionIncrementAttribute));
+        private readonly KeywordAttribute keywordAtt = addAttribute(typeof(KeywordAttribute));
+        private readonly OffsetAttribute offsetAtt = addAttribute(typeof(OffsetAttribute));
+        private readonly CharArraySet stopWords;
+
+        private State endState;
+
+        /// <summary>
+        /// Sole constructor. </summary>
+        public SuggestStopFilter(TokenStream input, CharArraySet stopWords)
+            : base(input)
+        {
+            this.stopWords = stopWords;
+        }
+
+        public override void Reset()
+        {
+            base.Reset();
+            endState = null;
+        }
+
+        public override void End()
+        {
+            if (endState == null)
+            {
+                base.End();
+            }
+            else
+            {
+                // NOTE: we already called .end() from our .next() when
+                // the stream was complete, so we do not call
+                // super.end() here
+                RestoreState(endState);
+            }
+        }
+
+        public override bool IncrementToken()
+        {
+            if (endState != null)
+            {
+                return false;
+            }
+
+            if (!Input.IncrementToken())
+            {
+                return false;
+            }
+
+            int skippedPositions = 0;
+            while (true)
+            {
+                if (stopWords.Contains(termAtt.Buffer(), 0, termAtt.Length))
+                {
+                    int posInc = posIncAtt.PositionIncrement;
+                    int endOffset = offsetAtt.EndOffset();
+                    // This token may be a stopword, if it's not end:
+                    State sav = CaptureState();
+                    if (Input.IncrementToken())
+                    {
+                        // It was a stopword; skip it
+                        skippedPositions += posInc;
+                    }
+                    else
+                    {
+                        ClearAttributes();
+                        Input.End();
+                        endState = CaptureState();
+                        int finalEndOffset = offsetAtt.EndOffset();
+                        Debug.Assert(finalEndOffset >= endOffset);
+                        if (finalEndOffset > endOffset)
+                        {
+                            // OK there was a token separator after the
+                            // stopword, so it was a stopword
+                            return false;
+                        }
+                        else
+                        {
+                            // No token separator after final token that
+                            // looked like a stop-word; don't filter it:
+                            RestoreState(sav);
+                            posIncAtt.PositionIncrement = skippedPositions + posIncAtt.PositionIncrement;
+                            keywordAtt.Keyword = true;
+                            return true;
+                        }
+                    }
+                }
+                else
+                {
+                    // Not a stopword; return the current token:
+                    posIncAtt.PositionIncrement = skippedPositions + posIncAtt.PositionIncrement;
+                    return true;
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/BufferedInputIterator.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/BufferedInputIterator.cs b/src/Lucene.Net.Suggest/Suggest/BufferedInputIterator.cs
new file mode 100644
index 0000000..916c41c
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/BufferedInputIterator.cs
@@ -0,0 +1,139 @@
+using System.Collections.Generic;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Search.Suggest
+{
+
+    /*
+     * 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.
+     */
+
+    /// <summary>
+    /// This wrapper buffers incoming elements.
+    /// @lucene.experimental
+    /// </summary>
+    public class BufferedInputIterator : InputIterator
+    {
+        // TODO keep this for now
+        /// <summary>
+        /// buffered term entries </summary>
+        protected internal BytesRefArray entries = new BytesRefArray(Counter.NewCounter());
+        /// <summary>
+        /// buffered payload entries </summary>
+        protected internal BytesRefArray payloads = new BytesRefArray(Counter.NewCounter());
+        /// <summary>
+        /// buffered context set entries </summary>
+        protected internal IList<HashSet<BytesRef>> contextSets = new List<HashSet<BytesRef>>();
+        /// <summary>
+        /// current buffer position </summary>
+        protected internal int curPos = -1;
+        /// <summary>
+        /// buffered weights, parallel with <seealso cref="#entries"/> </summary>
+        protected internal long[] freqs = new long[1];
+        private readonly BytesRef spare = new BytesRef();
+        private readonly BytesRef payloadSpare = new BytesRef();
+        private readonly bool hasPayloads;
+        private readonly IComparer<BytesRef> comp;
+
+        private readonly bool hasContexts_Renamed;
+
+        /// <summary>
+        /// Creates a new iterator, buffering entries from the specified iterator </summary>
+        public BufferedInputIterator(InputIterator source)
+        {
+            BytesRef spare;
+            int freqIndex = 0;
+            hasPayloads = source.HasPayloads;
+            hasContexts_Renamed = source.HasContexts;
+            while ((spare = source.Next()) != null)
+            {
+                entries.Append(spare);
+                if (hasPayloads)
+                {
+                    payloads.Append(source.Payload);
+                }
+                if (hasContexts_Renamed)
+                {
+                    contextSets.Add(source.Contexts);
+                }
+                if (freqIndex >= freqs.Length)
+                {
+                    freqs = ArrayUtil.Grow(freqs, freqs.Length + 1);
+                }
+                freqs[freqIndex++] = source.Weight;
+            }
+            comp = source.Comparator;
+        }
+
+        public virtual long Weight
+        {
+            get { return freqs[curPos]; }
+        }
+
+        public BytesRef Next()
+        {
+            if (++curPos < entries.Size())
+            {
+                entries.Get(spare, curPos);
+                return spare;
+            }
+            return null;
+        }
+
+        public virtual BytesRef Payload
+        {
+            get
+            {
+                if (hasPayloads && curPos < payloads.Size())
+                {
+                    return payloads.Get(payloadSpare, curPos);
+                }
+                return null;
+            }
+        }
+
+        public virtual bool HasPayloads
+        {
+            get { return hasPayloads; }
+        }
+
+        public IComparer<BytesRef> Comparator
+        {
+            get
+            {
+                return comp;
+            }
+        }
+
+        public virtual HashSet<BytesRef> Contexts
+        {
+            get
+            {
+                if (hasContexts_Renamed && curPos < contextSets.Count)
+                {
+                    return contextSets[curPos];
+                }
+                return null;
+            }
+        }
+
+        public virtual bool HasContexts
+        {
+            get { return hasContexts_Renamed; }
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/BufferingTermFreqIteratorWrapper.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/BufferingTermFreqIteratorWrapper.cs b/src/Lucene.Net.Suggest/Suggest/BufferingTermFreqIteratorWrapper.cs
new file mode 100644
index 0000000..2121a35
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/BufferingTermFreqIteratorWrapper.cs
@@ -0,0 +1,89 @@
+using System.Collections.Generic;
+using Lucene.Net.Search.Spell;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Search.Suggest
+{
+
+    /*
+     * 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.
+     */
+    /// <summary>
+    /// This wrapper buffers incoming elements.
+    /// @lucene.experimental
+    /// </summary>
+    public class BufferingTermFreqIteratorWrapper : TermFreqIterator
+    {
+        // TODO keep this for now
+        /// <summary>
+        /// buffered term entries </summary>
+        protected internal BytesRefArray entries = new BytesRefArray(Counter.NewCounter());
+        /// <summary>
+        /// current buffer position </summary>
+        protected internal int curPos = -1;
+        /// <summary>
+        /// buffered weights, parallel with <seealso cref="#entries"/> </summary>
+        protected internal long[] freqs = new long[1];
+        private readonly BytesRef spare = new BytesRef();
+        private readonly IComparer<BytesRef> comp;
+
+        /// <summary>
+        /// Creates a new iterator, buffering entries from the specified iterator
+        /// </summary>
+        public BufferingTermFreqIteratorWrapper(TermFreqIterator source)
+        {
+            this.comp = source.Comparator;
+            BytesRef spare;
+            int freqIndex = 0;
+            while ((spare = source.Next()) != null)
+            {
+                entries.Append(spare);
+                if (freqIndex >= freqs.Length)
+                {
+                    freqs = ArrayUtil.Grow(freqs, freqs.Length + 1);
+                }
+                freqs[freqIndex++] = source.Weight;
+            }
+
+        }
+
+        public virtual long Weight
+        {
+            get { return freqs[curPos]; }
+        }
+
+        public BytesRef Next()
+        {
+            if (++curPos < entries.Size())
+            {
+                entries.Get(spare, curPos);
+                return spare;
+            }
+            return null;
+        }
+
+        public IComparer<BytesRef> Comparator
+        {
+            get
+            {
+                return comp;
+            }
+        }
+
+
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/DocumentDictionary.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/DocumentDictionary.cs b/src/Lucene.Net.Suggest/Suggest/DocumentDictionary.cs
new file mode 100644
index 0000000..47cd026
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/DocumentDictionary.cs
@@ -0,0 +1,278 @@
+using System.Collections.Generic;
+using Lucene.Net.Documents;
+using Lucene.Net.Index;
+using Lucene.Net.Search.Spell;
+using Lucene.Net.Util;
+
+namespace Lucene.Net.Search.Suggest
+{
+
+    /*
+     * 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.
+     */
+    /// <summary>
+    /// <para>
+    /// Dictionary with terms, weights, payload (optional) and contexts (optional)
+    /// information taken from stored/indexed fields in a Lucene index.
+    /// </para>
+    /// <b>NOTE:</b> 
+    ///  <ul>
+    ///    <li>
+    ///      The term and (optionally) payload fields have to be
+    ///      stored
+    ///    </li>
+    ///    <li>
+    ///      The weight field can be stored or can be a <seealso cref="NumericDocValues"/>.
+    ///      If the weight field is not defined, the value of the weight is <code>0</code>
+    ///    </li>
+    ///    <li>
+    ///      if any of the term or (optionally) payload fields supplied
+    ///      do not have a value for a document, then the document is 
+    ///      skipped by the dictionary
+    ///    </li>
+    ///  </ul>
+    /// </summary>
+    public class DocumentDictionary : Dictionary
+    {
+
+        /// <summary>
+        /// <seealso cref="IndexReader"/> to load documents from </summary>
+        protected internal readonly IndexReader reader;
+
+        /// <summary>
+        /// Field to read payload from </summary>
+        protected internal readonly string payloadField;
+        /// <summary>
+        /// Field to read contexts from </summary>
+        protected internal readonly string contextsField;
+        private readonly string field;
+        private readonly string weightField;
+
+        /// <summary>
+        /// Creates a new dictionary with the contents of the fields named <code>field</code>
+        /// for the terms and <code>weightField</code> for the weights that will be used for
+        /// the corresponding terms.
+        /// </summary>
+        public DocumentDictionary(IndexReader reader, string field, string weightField)
+            : this(reader, field, weightField, null)
+        {
+        }
+
+        /// <summary>
+        /// Creates a new dictionary with the contents of the fields named <code>field</code>
+        /// for the terms, <code>weightField</code> for the weights that will be used for the 
+        /// the corresponding terms and <code>payloadField</code> for the corresponding payloads
+        /// for the entry.
+        /// </summary>
+        public DocumentDictionary(IndexReader reader, string field, string weightField, string payloadField)
+            : this(reader, field, weightField, payloadField, null)
+        {
+        }
+
+        /// <summary>
+        /// Creates a new dictionary with the contents of the fields named <code>field</code>
+        /// for the terms, <code>weightField</code> for the weights that will be used for the 
+        /// the corresponding terms, <code>payloadField</code> for the corresponding payloads
+        /// for the entry and <code>contextsFeild</code> for associated contexts.
+        /// </summary>
+        public DocumentDictionary(IndexReader reader, string field, string weightField, string payloadField, string contextsField)
+        {
+            this.reader = reader;
+            this.field = field;
+            this.weightField = weightField;
+            this.payloadField = payloadField;
+            this.contextsField = contextsField;
+        }
+
+        public virtual InputIterator EntryIterator
+        {
+            get
+            {
+                return new DocumentInputIterator(this, payloadField != null, contextsField != null);
+            }
+        }
+
+        /// <summary>
+        /// Implements <seealso cref="InputIterator"/> from stored fields. </summary>
+        protected internal class DocumentInputIterator : InputIterator
+        {
+            private readonly DocumentDictionary outerInstance;
+
+
+            internal readonly int docCount;
+            internal readonly HashSet<string> relevantFields;
+            internal readonly bool hasPayloads;
+            internal readonly bool hasContexts;
+            internal readonly Bits liveDocs;
+            internal int currentDocId = -1;
+            internal long currentWeight;
+            internal BytesRef currentPayload;
+            internal HashSet<BytesRef> currentContexts;
+            internal readonly NumericDocValues weightValues;
+
+
+            /// <summary>
+            /// Creates an iterator over term, weight and payload fields from the lucene
+            /// index. setting <code>withPayload</code> to false, implies an iterator
+            /// over only term and weight.
+            /// </summary>
+            public DocumentInputIterator(DocumentDictionary outerInstance, bool hasPayloads, bool hasContexts)
+            {
+                this.outerInstance = outerInstance;
+                this.hasPayloads = hasPayloads;
+                this.hasContexts = hasContexts;
+                docCount = outerInstance.reader.MaxDoc() - 1;
+                weightValues = (outerInstance.weightField != null) ? MultiDocValues.GetNumericValues(outerInstance.reader, outerInstance.weightField) : null;
+                liveDocs = (outerInstance.reader.Leaves().Count > 0) ? MultiFields.GetLiveDocs(outerInstance.reader) : null;
+                relevantFields = GetRelevantFields(new string[] { outerInstance.field, outerInstance.weightField, outerInstance.payloadField, outerInstance.contextsField });
+            }
+
+            public virtual long Weight
+            {
+                get { return currentWeight; }
+            }
+
+            public IComparer<BytesRef> Comparator
+            {
+                get
+                {
+                    return null;
+                }
+            }
+
+            public BytesRef Next()
+            {
+                while (currentDocId < docCount)
+                {
+                    currentDocId++;
+                    if (liveDocs != null && !liveDocs.Get(currentDocId))
+                    {
+                        continue;
+                    }
+
+                    Document doc = outerInstance.reader.Document(currentDocId, relevantFields);
+
+                    BytesRef tempPayload = null;
+                    BytesRef tempTerm = null;
+                    HashSet<BytesRef> tempContexts = new HashSet<BytesRef>();
+
+                    if (hasPayloads)
+                    {
+                        IndexableField payload = doc.GetField(outerInstance.payloadField);
+                        if (payload == null || (payload.BinaryValue() == null && payload.StringValue == null))
+                        {
+                            continue;
+                        }
+                        tempPayload = payload.BinaryValue() ?? new BytesRef(payload.StringValue);
+                    }
+
+                    if (hasContexts)
+                    {
+                        IndexableField[] contextFields = doc.GetFields(outerInstance.contextsField);
+                        foreach (IndexableField contextField in contextFields)
+                        {
+                            if (contextField.BinaryValue() == null && contextField.StringValue == null)
+                            {
+                                continue;
+                            }
+                            else
+                            {
+                                tempContexts.Add(contextField.BinaryValue() ?? new BytesRef(contextField.StringValue));
+                            }
+                        }
+                    }
+
+                    IndexableField fieldVal = doc.GetField(outerInstance.field);
+                    if (fieldVal == null || (fieldVal.BinaryValue() == null && fieldVal.StringValue == null))
+                    {
+                        continue;
+                    }
+                    tempTerm = (fieldVal.StringValue != null) ? new BytesRef(fieldVal.StringValue) : fieldVal.BinaryValue();
+
+                    currentPayload = tempPayload;
+                    currentContexts = tempContexts;
+                    currentWeight = GetWeight(doc, currentDocId);
+
+                    return tempTerm;
+                }
+                return null;
+            }
+
+            public virtual BytesRef Payload
+            {
+                get { return currentPayload; }
+            }
+
+            public virtual bool HasPayloads
+            {
+                get { return hasPayloads; }
+            }
+
+            /// <summary>
+            /// Returns the value of the <code>weightField</code> for the current document.
+            /// Retrieves the value for the <code>weightField</code> if its stored (using <code>doc</code>)
+            /// or if its indexed as <seealso cref="NumericDocValues"/> (using <code>docId</code>) for the document.
+            /// If no value is found, then the weight is 0.
+            /// </summary>
+            protected internal virtual long GetWeight(Document doc, int docId)
+            {
+                IndexableField weight = doc.GetField(outerInstance.weightField);
+                if (weight != null) // found weight as stored
+                {
+                    return (weight.NumericValue != null) ? (long)weight.NumericValue : 0;
+                } // found weight as NumericDocValue
+                else if (weightValues != null)
+                {
+                    return weightValues.Get(docId);
+                } // fall back
+                else
+                {
+                    return 0;
+                }
+            }
+
+            internal HashSet<string> GetRelevantFields(params string[] fields)
+            {
+                var relevantFields = new HashSet<string>();
+                foreach (string relevantField in fields)
+                {
+                    if (relevantField != null)
+                    {
+                        relevantFields.Add(relevantField);
+                    }
+                }
+                return relevantFields;
+            }
+
+            public virtual HashSet<BytesRef> Contexts
+            {
+                get
+                {
+                    if (hasContexts)
+                    {
+                        return currentContexts;
+                    }
+                    return null;
+                }
+            }
+
+            public virtual bool HasContexts
+            {
+                get { return hasContexts; }
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/0ebac726/src/Lucene.Net.Suggest/Suggest/DocumentValueSourceDictionary.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Suggest/Suggest/DocumentValueSourceDictionary.cs b/src/Lucene.Net.Suggest/Suggest/DocumentValueSourceDictionary.cs
new file mode 100644
index 0000000..5793c44
--- /dev/null
+++ b/src/Lucene.Net.Suggest/Suggest/DocumentValueSourceDictionary.cs
@@ -0,0 +1,169 @@
+using System;
+using System.Collections.Generic;
+using System.IO;
+using Lucene.Net.Documents;
+using Lucene.Net.Index;
+
+namespace Lucene.Net.Search.Suggest
+{
+
+    /*
+     * 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.
+     */
+    /// <summary>
+    /// <para>
+    /// Dictionary with terms and optionally payload information 
+    /// taken from stored fields in a Lucene index. Similar to 
+    /// <seealso cref="DocumentDictionary"/>, except it obtains the weight
+    /// of the terms in a document based on a <seealso cref="ValueSource"/>.
+    /// </para>
+    /// <b>NOTE:</b> 
+    ///  <ul>
+    ///    <li>
+    ///      The term and (optionally) payload fields have to be
+    ///      stored
+    ///    </li>
+    ///    <li>
+    ///      if the term or (optionally) payload fields supplied
+    ///      do not have a value for a document, then the document is 
+    ///      rejected by the dictionary
+    ///    </li>
+    ///  </ul>
+    ///  <para>
+    ///  In practice the <seealso cref="ValueSource"/> will likely be obtained
+    ///  using the lucene expression module. The following example shows
+    ///  how to create a <seealso cref="ValueSource"/> from a simple addition of two
+    ///  fields:
+    ///  <code>
+    ///    Expression expression = JavascriptCompiler.compile("f1 + f2");
+    ///    SimpleBindings bindings = new SimpleBindings();
+    ///    bindings.add(new SortField("f1", SortField.Type.LONG));
+    ///    bindings.add(new SortField("f2", SortField.Type.LONG));
+    ///    ValueSource valueSource = expression.getValueSource(bindings);
+    ///  </code>
+    ///  </para>
+    /// 
+    /// </summary>
+    public class DocumentValueSourceDictionary : DocumentDictionary
+    {
+
+        private readonly ValueSource weightsValueSource;
+
+        /// <summary>
+        /// Creates a new dictionary with the contents of the fields named <code>field</code>
+        /// for the terms, <code>payload</code> for the corresponding payloads, <code>contexts</code>
+        /// for the associated contexts and uses the <code>weightsValueSource</code> supplied 
+        /// to determine the score.
+        /// </summary>
+        public DocumentValueSourceDictionary(IndexReader reader, string field, ValueSource weightsValueSource, string payload, string contexts)
+            : base(reader, field, null, payload, contexts)
+        {
+            this.weightsValueSource = weightsValueSource;
+        }
+        /// <summary>
+        /// Creates a new dictionary with the contents of the fields named <code>field</code>
+        /// for the terms, <code>payloadField</code> for the corresponding payloads
+        /// and uses the <code>weightsValueSource</code> supplied to determine the 
+        /// score.
+        /// </summary>
+        public DocumentValueSourceDictionary(IndexReader reader, string field, ValueSource weightsValueSource, string payload)
+            : base(reader, field, null, payload)
+        {
+            this.weightsValueSource = weightsValueSource;
+        }
+
+        /// <summary>
+        /// Creates a new dictionary with the contents of the fields named <code>field</code>
+        /// for the terms and uses the <code>weightsValueSource</code> supplied to determine the 
+        /// score.
+        /// </summary>
+        public DocumentValueSourceDictionary(IndexReader reader, string field, ValueSource weightsValueSource)
+            : base(reader, field, null, null)
+        {
+            this.weightsValueSource = weightsValueSource;
+        }
+
+        public override InputIterator EntryIterator
+        {
+            get
+            {
+                return new DocumentValueSourceInputIterator(this, payloadField != null, contextsField != null);
+            }
+        }
+
+        internal sealed class DocumentValueSourceInputIterator : DocumentDictionary.DocumentInputIterator
+        {
+            private readonly DocumentValueSourceDictionary outerInstance;
+
+
+            internal FunctionValues currentWeightValues;
+            /// <summary>
+            /// leaves of the reader </summary>
+            internal readonly IList<AtomicReaderContext> leaves;
+            /// <summary>
+            /// starting docIds of all the leaves </summary>
+            internal readonly int[] starts;
+            /// <summary>
+            /// current leave index </summary>
+            internal int currentLeafIndex = 0;
+
+            //JAVA TO C# CONVERTER WARNING: Method 'throws' clauses are not available in .NET:
+            //ORIGINAL LINE: public DocumentValueSourceInputIterator(boolean hasPayloads, boolean hasContexts) throws java.io.IOException
+            public DocumentValueSourceInputIterator(DocumentValueSourceDictionary outerInstance, bool hasPayloads, bool hasContexts)
+                : base(outerInstance, hasPayloads, hasContexts)
+            {
+                this.outerInstance = outerInstance;
+                leaves = outerInstance.reader.Leaves();
+                starts = new int[leaves.Count + 1];
+                for (int i = 0; i < leaves.Count; i++)
+                {
+                    starts[i] = leaves[i].DocBase;
+                }
+                starts[leaves.Count] = outerInstance.reader.MaxDoc();
+                currentWeightValues = (leaves.Count > 0) ? outerInstance.weightsValueSource.GetValues(new Dictionary<string, object>(), leaves[currentLeafIndex]) : null;
+            }
+
+            /// <summary>
+            /// Returns the weight for the current <code>docId</code> as computed 
+            /// by the <code>weightsValueSource</code>
+            /// 
+            /// </summary>
+            protected internal override long GetWeight(Document doc, int docId)
+            {
+                if (currentWeightValues == null)
+                {
+                    return 0;
+                }
+                int subIndex = ReaderUtil.subIndex(docId, starts);
+                if (subIndex != currentLeafIndex)
+                {
+                    currentLeafIndex = subIndex;
+                    try
+                    {
+                        currentWeightValues = outerInstance.weightsValueSource.GetValues(new Dictionary<string, object>(), leaves[currentLeafIndex]);
+                    }
+                    catch (IOException)
+                    {
+                        throw new Exception();
+                    }
+                }
+                return currentWeightValues.LongVal(docId - starts[subIndex]);
+            }
+
+        }
+    }
+
+}
\ No newline at end of file