diff --git lucene/suggest/src/java/org/apache/lucene/search/spell/Dictionary.java lucene/suggest/src/java/org/apache/lucene/search/spell/Dictionary.java
index d1ed4e7..59194b8 100644
--- lucene/suggest/src/java/org/apache/lucene/search/spell/Dictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/spell/Dictionary.java
@@ -17,19 +17,20 @@ package org.apache.lucene.search.spell;
  */
 
 import java.io.IOException;
-import org.apache.lucene.util.BytesRefIterator;
+
+import org.apache.lucene.search.suggest.InputIterator;
 
 /**
  * A simple interface representing a Dictionary. A Dictionary
- * here is just a list of words.
+ * here is a list of entries, where every entry consists of
+ * term, weight and payload.
  * 
- *
  */
 public interface Dictionary {
 
   /**
-   * Return all words present in the dictionary
+   * Returns an iterator over all the entries
    * @return Iterator
    */
-  BytesRefIterator getWordsIterator() throws IOException;
+  InputIterator getEntryIterator() throws IOException;
 }
diff --git lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java
index 826ba28..0150ed1 100644
--- lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/spell/HighFrequencyDictionary.java
@@ -56,7 +56,7 @@ public class HighFrequencyDictionary implements Dictionary {
   }
 
   @Override
-  public final BytesRefIterator getWordsIterator() throws IOException {
+  public final InputIterator getEntryIterator() throws IOException {
     return new HighFrequencyIterator();
   }
 
diff --git lucene/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java lucene/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
index b5d6627..d55e5bb 100644
--- lucene/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search.spell;
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.MultiFields;
@@ -42,12 +43,12 @@ public class LuceneDictionary implements Dictionary {
   }
 
   @Override
-  public final BytesRefIterator getWordsIterator() throws IOException {
+  public final InputIterator getEntryIterator() throws IOException {
     final Terms terms = MultiFields.getTerms(reader, field);
     if (terms != null) {
-      return terms.iterator(null);
+      return new InputIterator.InputIteratorWrapper(terms.iterator(null));
     } else {
-      return BytesRefIterator.EMPTY;
+      return InputIterator.EMPTY;
     }
   }
 }
diff --git lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
index 7071ff7..5e77021 100644
--- lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/spell/PlainTextDictionary.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
 
+import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefIterator;
 import org.apache.lucene.util.IOUtils;
@@ -66,8 +67,8 @@ public class PlainTextDictionary implements Dictionary {
   }
 
   @Override
-  public BytesRefIterator getWordsIterator() throws IOException {
-    return new FileIterator();
+  public InputIterator getEntryIterator() throws IOException {
+    return new InputIterator.InputIteratorWrapper(new FileIterator());
   }
 
   final class FileIterator implements BytesRefIterator {
diff --git lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java
index 6f5f399..e61a287 100644
--- lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java
+++ lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java
@@ -512,7 +512,7 @@ public class SpellChecker implements java.io.Closeable {
       boolean isEmpty = termsEnums.isEmpty();
 
       try { 
-        BytesRefIterator iter = dict.getWordsIterator();
+        BytesRefIterator iter = dict.getEntryIterator();
         BytesRef currentTerm;
         
         terms: while ((currentTerm = iter.next()) != null) {
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
index d948e20..91fdf24 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentDictionary.java
@@ -86,7 +86,7 @@ public class DocumentDictionary implements Dictionary {
   }
   
   @Override
-  public BytesRefIterator getWordsIterator() throws IOException {
+  public InputIterator getEntryIterator() throws IOException {
     return new DocumentInputIterator(payloadField!=null);
   }
 
@@ -102,7 +102,6 @@ public class DocumentDictionary implements Dictionary {
     private BytesRef currentPayload = null;
     private final NumericDocValues weightValues;
     
-    
     /**
      * Creates an iterator over term, weight and payload fields from the lucene
      * index. setting <code>withPayload</code> to false, implies an iterator
@@ -150,7 +149,7 @@ public class DocumentDictionary implements Dictionary {
         
         currentPayload = tempPayload;
         currentWeight = getWeight(doc, currentDocId);
-
+        
         return tempTerm;
       }
       return null;
@@ -165,7 +164,7 @@ public class DocumentDictionary implements Dictionary {
     public boolean hasPayloads() {
       return hasPayloads;
     }
-
+    
     /** 
      * 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>)
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
index cbc5763..d5f720e 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/DocumentValueSourceDictionary.java
@@ -92,7 +92,7 @@ public class DocumentValueSourceDictionary extends DocumentDictionary {
   }
   
   @Override
-  public BytesRefIterator getWordsIterator() throws IOException {
+  public InputIterator getEntryIterator() throws IOException {
     return new DocumentValueSourceInputIterator(payloadField!=null);
   }
   
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java
index 5e59685..28921be 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java
@@ -106,7 +106,7 @@ public class FileDictionary implements Dictionary {
   }
 
   @Override
-  public InputIterator getWordsIterator() {
+  public InputIterator getEntryIterator() {
     try {
       return new FileIterator();
     } catch (IOException e) {
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/InputIterator.java lucene/suggest/src/java/org/apache/lucene/search/suggest/InputIterator.java
index bda1332..c98825d 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/InputIterator.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/InputIterator.java
@@ -44,6 +44,9 @@ public interface InputIterator extends BytesRefIterator {
   /** Returns true if the iterator has payloads */
   public boolean hasPayloads();
   
+  /** Singleton InputIterator that iterates over 0 BytesRefs. */
+  public static final InputIterator EMPTY = new InputIteratorWrapper(BytesRefIterator.EMPTY);
+  
   /**
    * Wraps a BytesRefIterator as a suggester InputIterator, with all weights
    * set to <code>1</code> and carries no payload
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
index 3b4e09c..aed404a 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/Lookup.java
@@ -24,8 +24,12 @@ import java.util.Comparator;
 import java.util.List;
 
 import org.apache.lucene.search.spell.Dictionary;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PriorityQueue;
 
 /**
@@ -33,6 +37,12 @@ import org.apache.lucene.util.PriorityQueue;
  * @lucene.experimental
  */
 public abstract class Lookup {
+  
+  /**
+   * Number of entries the lookup was built with
+   */
+  private int count = 0;
+
   /**
    * Result of a lookup.
    */
@@ -155,23 +165,62 @@ public abstract class Lookup {
    * or unsorted keys from the dictionary's iterator - use
    * {@link SortedInputIterator} or
    * {@link UnsortedInputIterator} in such case.
+   * 
+   * Returns number of entries used to build the lookup
    */
   public void build(Dictionary dict) throws IOException {
-    BytesRefIterator it = dict.getWordsIterator();
-    InputIterator tfit;
-    if (it instanceof InputIterator) {
-      tfit = (InputIterator)it;
-    } else {
-      tfit = new InputIterator.InputIteratorWrapper(it);
+    count = build(dict.getEntryIterator());
+  }
+  
+  /**
+   * Discard current lookup data and load it from a previously saved copy,
+   * along with Lookup metadata (e.g. total entries count)
+   * Optional operation.
+   * @param input the {@link DataInput} to load the lookup data.
+   * @return true if completed successfully, false if unsuccessful or not supported.
+   * @throws IOException when fatal IO error occurs.
+   */
+  public boolean load(InputStream input) throws IOException {
+    DataInput dataIn = new InputStreamDataInput(input);
+    try {
+      count = dataIn.readVInt();
+      return load(dataIn);
+    } finally {
+      IOUtils.close(input);
     }
-    build(tfit);
+  }
+  
+  /**
+   * Persist the constructed lookup data to a directory along
+   * with Lookup metadata (e.g. total entries count)
+   * @param output {@link OutputStream} 
+   * @return true if successful, false if unsuccessful or not supported.
+   * @throws IOException when fatal IO error occurs.
+   */
+  public boolean store(OutputStream output) throws IOException {
+    DataOutput dataOut = new OutputStreamDataOutput(output);
+    try {
+      dataOut.writeVInt(count);
+      return store(dataOut);
+    } finally {
+      IOUtils.close(output);
+    }
+  }
+  
+  /**
+   * Returns the number of entries the lookup was built with
+   */
+  public int getCount() {
+    return count;
   }
   
   /**
    * Builds up a new internal {@link Lookup} representation based on the given {@link InputIterator}.
    * The implementation might re-sort the data internally.
+   *
+   * Returns number of entries used to build the lookup
    */
-  public abstract void build(InputIterator tfit) throws IOException;
+  public abstract int build(InputIterator inputIterator) throws IOException;
   
   /**
    * Look up a key and return possible completion for this key.
@@ -183,23 +232,22 @@ public abstract class Lookup {
    */
   public abstract List<LookupResult> lookup(CharSequence key, boolean onlyMorePopular, int num);
 
-  
   /**
    * Persist the constructed lookup data to a directory. Optional operation.
-   * @param output {@link OutputStream} to write the data to.
+   * @param output {@link DataOutput} to write the data to.
    * @return true if successful, false if unsuccessful or not supported.
    * @throws IOException when fatal IO error occurs.
    */
-  public abstract boolean store(OutputStream output) throws IOException;
+  public abstract boolean store(DataOutput output) throws IOException;
 
   /**
    * Discard current lookup data and load it from a previously saved copy.
    * Optional operation.
-   * @param input the {@link InputStream} to load the lookup data.
+   * @param input the {@link DataInput} to load the lookup data.
    * @return true if completed successfully, false if unsuccessful or not supported.
    * @throws IOException when fatal IO error occurs.
    */
-  public abstract boolean load(InputStream input) throws IOException;
+  public abstract boolean load(DataInput input) throws IOException;
   
   /**
    * Get the size of the underlying lookup implementation in memory
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
index 0cf4212..427e87c 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
@@ -68,6 +68,8 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.suggest.Lookup.LookupResult; // javadocs
 import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.search.suggest.Lookup;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.BytesRef;
@@ -180,7 +182,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
   }
 
   @Override
-  public void build(InputIterator iter) throws IOException {
+  public int build(InputIterator iter) throws IOException {
 
     if (searcher != null) {
       searcher.getIndexReader().close();
@@ -194,6 +196,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
     IndexWriter w2 = null;
     AtomicReader r = null;
     boolean success = false;
+    int count = 0;
     try {
       Analyzer gramAnalyzer = new AnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) {
           @Override
@@ -239,7 +242,6 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
       } else {
         payloadField = null;
       }
-
       //long t0 = System.nanoTime();
       while ((text = iter.next()) != null) {
         String textString = text.utf8ToString();
@@ -251,6 +253,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
           payloadField.setBytesValue(iter.payload());
         }
         w.addDocument(doc);
+        count++;
       }
       //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
 
@@ -314,6 +317,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
         IOUtils.closeWhileHandlingException(w, w2, r, dirTmp);
       }
     }
+    return count;
   }
 
   /**
@@ -612,12 +616,12 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
   }
 
   @Override
-  public boolean store(OutputStream out) {
+  public boolean store(DataOutput out) {
     return false;
   }
 
   @Override
-  public boolean load(InputStream out) {
+  public boolean load(DataInput out) {
     return false;
   }
 
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
index 4278440..43bd8e0 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
@@ -19,8 +19,6 @@ package org.apache.lucene.search.suggest.analyzing;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -38,8 +36,6 @@ import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
@@ -379,7 +375,7 @@ public class AnalyzingSuggester extends Lookup {
   }
 
   @Override
-  public void build(InputIterator iterator) throws IOException {
+  public int build(InputIterator iterator) throws IOException {
     String prefix = getClass().getSimpleName();
     File directory = Sort.defaultTempDir();
     File tempInput = File.createTempFile(prefix, ".input", directory);
@@ -394,6 +390,7 @@ public class AnalyzingSuggester extends Lookup {
     TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
 
     boolean success = false;
+    int count = 0;
     byte buffer[] = new byte[8];
     try {
       ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
@@ -458,6 +455,7 @@ public class AnalyzingSuggester extends Lookup {
 
           writer.write(buffer, 0, output.getPosition());
         }
+        count++;
       }
       writer.close();
 
@@ -568,35 +566,26 @@ public class AnalyzingSuggester extends Lookup {
       tempInput.delete();
       tempSorted.delete();
     }
+    return count;
   }
 
   @Override
-  public boolean store(OutputStream output) throws IOException {
-    DataOutput dataOut = new OutputStreamDataOutput(output);
-    try {
-      if (fst == null) {
-        return false;
-      }
-
-      fst.save(dataOut);
-      dataOut.writeVInt(maxAnalyzedPathsForOneInput);
-      dataOut.writeByte((byte) (hasPayloads ? 1 : 0));
-    } finally {
-      IOUtils.close(output);
+  public boolean store(DataOutput output) throws IOException {
+    if (fst == null) {
+      return false;
     }
+
+    fst.save(output);
+    output.writeVInt(maxAnalyzedPathsForOneInput);
+    output.writeByte((byte) (hasPayloads ? 1 : 0));
     return true;
   }
 
   @Override
-  public boolean load(InputStream input) throws IOException {
-    DataInput dataIn = new InputStreamDataInput(input);
-    try {
-      this.fst = new FST<Pair<Long,BytesRef>>(dataIn, new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
-      maxAnalyzedPathsForOneInput = dataIn.readVInt();
-      hasPayloads = dataIn.readByte() == 1;
-    } finally {
-      IOUtils.close(input);
-    }
+  public boolean load(DataInput input) throws IOException {
+    this.fst = new FST<Pair<Long,BytesRef>>(input, new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
+    maxAnalyzedPathsForOneInput = input.readVInt();
+    hasPayloads = input.readByte() == 1;
     return true;
   }
 
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
index e901ef7..6aa7950 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
@@ -22,8 +22,6 @@ package org.apache.lucene.search.suggest.analyzing;
 //   - add pruning of low-freq ngrams?
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 //import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -62,8 +60,6 @@ import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
@@ -274,14 +270,14 @@ public class FreeTextSuggester extends Lookup {
   }
 
   @Override
-  public void build(InputIterator iterator) throws IOException {
-    build(iterator, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB);
+  public int build(InputIterator iterator) throws IOException {
+    return build(iterator, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB);
   }
 
   /** Build the suggest index, using up to the specified
    *  amount of temporary RAM while building.  Note that
    *  the weights for the suggestions are ignored. */
-  public void build(InputIterator iterator, double ramBufferSizeMB) throws IOException {
+  public int build(InputIterator iterator, double ramBufferSizeMB) throws IOException {
     if (iterator.hasPayloads()) {
       throw new IllegalArgumentException("payloads are not supported");
     }
@@ -320,6 +316,7 @@ public class FreeTextSuggester extends Lookup {
     IndexReader reader = null;
 
     boolean success = false;
+    int count = 0;
     try {
       while (true) {
         BytesRef surfaceForm = iterator.next();
@@ -328,6 +325,7 @@ public class FreeTextSuggester extends Lookup {
         }
         field.setStringValue(surfaceForm.utf8ToString());
         writer.addDocument(doc);
+        count++;
       }
       reader = DirectoryReader.open(writer, false);
 
@@ -394,34 +392,33 @@ public class FreeTextSuggester extends Lookup {
         dir.close();
       }
     }
+    return count;
   }
 
   @Override
-  public boolean store(OutputStream output) throws IOException {
-    DataOutput out = new OutputStreamDataOutput(output);
-    CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
-    out.writeByte(separator);
-    out.writeVInt(grams);
-    out.writeVLong(totTokens);
-    fst.save(out);
+  public boolean store(DataOutput output) throws IOException {
+    CodecUtil.writeHeader(output, CODEC_NAME, VERSION_CURRENT);
+    output.writeByte(separator);
+    output.writeVInt(grams);
+    output.writeVLong(totTokens);
+    fst.save(output);
     return true;
   }
 
   @Override
-  public boolean load(InputStream input) throws IOException {
-    DataInput in = new InputStreamDataInput(input);
-    CodecUtil.checkHeader(in, CODEC_NAME, VERSION_START, VERSION_START);
-    byte separatorOrig = in.readByte();
+  public boolean load(DataInput input) throws IOException {
+    CodecUtil.checkHeader(input, CODEC_NAME, VERSION_START, VERSION_START);
+    byte separatorOrig = input.readByte();
     if (separatorOrig != separator) {
       throw new IllegalStateException("separator=" + separator + " is incorrect: original model was built with separator=" + separatorOrig);
     }
-    int gramsOrig = in.readVInt();
+    int gramsOrig = input.readVInt();
     if (gramsOrig != grams) {
       throw new IllegalStateException("grams=" + grams + " is incorrect: original model was built with grams=" + gramsOrig);
     }
-    totTokens = in.readVLong();
+    totTokens = input.readVLong();
 
-    fst = new FST<Long>(in, PositiveIntOutputs.getSingleton());
+    fst = new FST<Long>(input, PositiveIntOutputs.getSingleton());
 
     return true;
   }
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
index 7c88a3e..c5fa7da 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
@@ -32,6 +32,8 @@ import org.apache.lucene.search.suggest.fst.FSTCompletion.Completion;
 import org.apache.lucene.search.suggest.tst.TSTLookup;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.*;
@@ -140,8 +142,8 @@ public class FSTCompletionLookup extends Lookup {
   }
 
   @Override
-  public void build(InputIterator tfit) throws IOException {
-    if (tfit.hasPayloads()) {
+  public int build(InputIterator iterator) throws IOException {
+    if (iterator.hasPayloads()) {
       throw new IllegalArgumentException("this suggester doesn't support payloads");
     }
     File tempInput = File.createTempFile(
@@ -156,17 +158,18 @@ public class FSTCompletionLookup extends Lookup {
     // Push floats up front before sequences to sort them. For now, assume they are non-negative.
     // If negative floats are allowed some trickery needs to be done to find their byte order.
     boolean success = false;
+    int count = 0;
     try {
       byte [] buffer = new byte [0];
       ByteArrayDataOutput output = new ByteArrayDataOutput(buffer);
       BytesRef spare;
-      while ((spare = tfit.next()) != null) {
+      while ((spare = iterator.next()) != null) {
         if (spare.length + 4 >= buffer.length) {
           buffer = ArrayUtil.grow(buffer, spare.length + 4);
         }
 
         output.reset(buffer);
-        output.writeInt(encodeWeight(tfit.weight()));
+        output.writeInt(encodeWeight(iterator.weight()));
         output.writeBytes(spare.bytes, spare.offset, spare.length);
         writer.write(buffer, 0, output.getPosition());
       }
@@ -207,6 +210,7 @@ public class FSTCompletionLookup extends Lookup {
         builder.add(tmp2, bucket);
 
         line++;
+        count++;
       }
 
       // The two FSTCompletions share the same automaton.
@@ -224,6 +228,7 @@ public class FSTCompletionLookup extends Lookup {
       tempInput.delete();
       tempSorted.delete();
     }
+    return count;
   }
   
   /** weight -> cost */
@@ -264,28 +269,19 @@ public class FSTCompletionLookup extends Lookup {
 
 
   @Override
-  public synchronized boolean store(OutputStream output) throws IOException {
-
-    try {
-      if (this.normalCompletion == null || normalCompletion.getFST() == null) 
-        return false;
-      normalCompletion.getFST().save(new OutputStreamDataOutput(output));
-    } finally {
-      IOUtils.close(output);
-    }
+  public synchronized boolean store(DataOutput output) throws IOException {
+    if (this.normalCompletion == null || normalCompletion.getFST() == null) 
+      return false;
+    normalCompletion.getFST().save(output);
     return true;
   }
 
   @Override
-  public synchronized boolean load(InputStream input) throws IOException {
-    try {
-      this.higherWeightsCompletion = new FSTCompletion(new FST<Object>(
-          new InputStreamDataInput(input), NoOutputs.getSingleton()));
-      this.normalCompletion = new FSTCompletion(
-          higherWeightsCompletion.getFST(), false, exactMatchFirst);
-    } finally {
-      IOUtils.close(input);
-    }
+  public synchronized boolean load(DataInput input) throws IOException {
+    this.higherWeightsCompletion = new FSTCompletion(new FST<Object>(
+        input, NoOutputs.getSingleton()));
+    this.normalCompletion = new FSTCompletion(
+        higherWeightsCompletion.getFST(), false, exactMatchFirst);
     return true;
   }
 
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
index eaff404..a4fe2c6 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
@@ -18,8 +18,6 @@ package org.apache.lucene.search.suggest.fst;
  */
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -31,12 +29,11 @@ import org.apache.lucene.search.suggest.Sort.ByteSequencesWriter;
 import org.apache.lucene.search.suggest.SortedInputIterator;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayDataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.fst.Builder;
@@ -92,10 +89,11 @@ public class WFSTCompletionLookup extends Lookup {
   }
   
   @Override
-  public void build(InputIterator iterator) throws IOException {
+  public int build(InputIterator iterator) throws IOException {
     if (iterator.hasPayloads()) {
       throw new IllegalArgumentException("this suggester doesn't support payloads");
     }
+    int count = 0;
     BytesRef scratch = new BytesRef();
     InputIterator iter = new WFSTInputIterator(iterator);
     IntsRef scratchInts = new IntsRef();
@@ -114,31 +112,25 @@ public class WFSTCompletionLookup extends Lookup {
       Util.toIntsRef(scratch, scratchInts);
       builder.add(scratchInts, cost);
       previous.copyBytes(scratch);
+      count++;
     }
     fst = builder.finish();
+    return count;
   }
 
   
   @Override
-  public boolean store(OutputStream output) throws IOException {
-    try {
-      if (fst == null) {
-        return false;
-      }
-      fst.save(new OutputStreamDataOutput(output));
-    } finally {
-      IOUtils.close(output);
+  public boolean store(DataOutput output) throws IOException {
+    if (fst == null) {
+      return false;
     }
+    fst.save(output);
     return true;
   }
 
   @Override
-  public boolean load(InputStream input) throws IOException {
-    try {
-      this.fst = new FST<Long>(new InputStreamDataInput(input), PositiveIntOutputs.getSingleton());
-    } finally {
-      IOUtils.close(input);
-    }
+  public boolean load(DataInput input) throws IOException {
+    this.fst = new FST<Long>(input, PositiveIntOutputs.getSingleton());
     return true;
   }
 
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java
index 65c4532..0f2c1a8 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java
@@ -17,20 +17,17 @@ package org.apache.lucene.search.suggest.jaspell;
  * limitations under the License.
  */
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellTernarySearchTrie.TSTNode;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -52,24 +49,27 @@ public class JaspellLookup extends Lookup {
   public JaspellLookup() {}
 
   @Override
-  public void build(InputIterator tfit) throws IOException {
-    if (tfit.hasPayloads()) {
+  public int build(InputIterator iterator) throws IOException {
+    if (iterator.hasPayloads()) {
       throw new IllegalArgumentException("this suggester doesn't support payloads");
     }
+    int count = 0;
     trie = new JaspellTernarySearchTrie();
     trie.setMatchAlmostDiff(editDistance);
     BytesRef spare;
     final CharsRef charsSpare = new CharsRef();
 
-    while ((spare = tfit.next()) != null) {
-      final long weight = tfit.weight();
+    while ((spare = iterator.next()) != null) {
+      final long weight = iterator.weight();
       if (spare.length == 0) {
         continue;
       }
       charsSpare.grow(spare.length);
       UnicodeUtil.UTF8toUTF16(spare.bytes, spare.offset, spare.length, charsSpare);
       trie.put(charsSpare.toString(), Long.valueOf(weight));
+      count++;
     }
+    return count;
   }
 
   /** 
@@ -131,8 +131,8 @@ public class JaspellLookup extends Lookup {
   private static final byte HI_KID = 0x04;
   private static final byte HAS_VALUE = 0x08;
  
-  private void readRecursively(DataInputStream in, TSTNode node) throws IOException {
-    node.splitchar = in.readChar();
+  private void readRecursively(DataInput in, TSTNode node) throws IOException {
+    node.splitchar = in.readString().charAt(0);
     byte mask = in.readByte();
     if ((mask & HAS_VALUE) != 0) {
       node.data = Long.valueOf(in.readLong());
@@ -154,11 +154,11 @@ public class JaspellLookup extends Lookup {
     }
   }
 
-  private void writeRecursively(DataOutputStream out, TSTNode node) throws IOException {
+  private void writeRecursively(DataOutput out, TSTNode node) throws IOException {
     if (node == null) {
       return;
     }
-    out.writeChar(node.splitchar);
+    out.writeString(new String(new char[] {node.splitchar}, 0, 1));
     byte mask = 0;
     if (node.relatives[TSTNode.LOKID] != null) mask |= LO_KID;
     if (node.relatives[TSTNode.EQKID] != null) mask |= EQ_KID;
@@ -174,31 +174,20 @@ public class JaspellLookup extends Lookup {
   }
 
   @Override
-  public boolean store(OutputStream output) throws IOException {
+  public boolean store(DataOutput output) throws IOException {
     TSTNode root = trie.getRoot();
     if (root == null) { // empty tree
       return false;
     }
-    DataOutputStream out = new DataOutputStream(output);
-    try {
-      writeRecursively(out, root);
-      out.flush();
-    } finally {
-      IOUtils.close(out);
-    }
+    writeRecursively(output, root);
     return true;
   }
 
   @Override
-  public boolean load(InputStream input) throws IOException {
-    DataInputStream in = new DataInputStream(input);
+  public boolean load(DataInput input) throws IOException {
     TSTNode root = trie.new TSTNode('\0', null);
-    try {
-      readRecursively(in, root);
-      trie.setRoot(root);
-    } finally {
-      IOUtils.close(in);
-    }
+    readRecursively(input, root);
+    trie.setRoot(root);
     return true;
   }
 
diff --git lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
index 59cd5f9..4a1da3b 100644
--- lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
+++ lucene/suggest/src/java/org/apache/lucene/search/suggest/tst/TSTLookup.java
@@ -17,20 +17,17 @@ package org.apache.lucene.search.suggest.tst;
  * limitations under the License.
  */
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.search.suggest.InputIterator;
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.SortedInputIterator;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -51,26 +48,28 @@ public class TSTLookup extends Lookup {
   public TSTLookup() {}
 
   @Override
-  public void build(InputIterator tfit) throws IOException {
-    if (tfit.hasPayloads()) {
+  public int build(InputIterator iterator) throws IOException {
+    if (iterator.hasPayloads()) {
       throw new IllegalArgumentException("this suggester doesn't support payloads");
     }
     root = new TernaryTreeNode();
 
     // make sure it's sorted and the comparator uses UTF16 sort order
-    tfit = new SortedInputIterator(tfit, BytesRef.getUTF8SortedAsUTF16Comparator());
-
+    iterator = new SortedInputIterator(iterator, BytesRef.getUTF8SortedAsUTF16Comparator());
+    int count = 0;
     ArrayList<String> tokens = new ArrayList<String>();
     ArrayList<Number> vals = new ArrayList<Number>();
     BytesRef spare;
     CharsRef charsSpare = new CharsRef();
-    while ((spare = tfit.next()) != null) {
+    while ((spare = iterator.next()) != null) {
       charsSpare.grow(spare.length);
       UnicodeUtil.UTF8toUTF16(spare.bytes, spare.offset, spare.length, charsSpare);
       tokens.add(charsSpare.toString());
-      vals.add(Long.valueOf(tfit.weight()));
+      vals.add(Long.valueOf(iterator.weight()));
+      count++;
     }
     autocomplete.balancedTree(tokens.toArray(), vals.toArray(), 0, tokens.size() - 1, root);
+    return count;
   }
 
   /** 
@@ -148,11 +147,11 @@ public class TSTLookup extends Lookup {
   private static final byte HAS_VALUE = 0x10;
 
   // pre-order traversal
-  private void readRecursively(DataInputStream in, TernaryTreeNode node) throws IOException {
-    node.splitchar = in.readChar();
+  private void readRecursively(DataInput in, TernaryTreeNode node) throws IOException {
+    node.splitchar = in.readString().charAt(0);
     byte mask = in.readByte();
     if ((mask & HAS_TOKEN) != 0) {
-      node.token = in.readUTF();
+      node.token = in.readString();
     }
     if ((mask & HAS_VALUE) != 0) {
       node.val = Long.valueOf(in.readLong());
@@ -172,9 +171,9 @@ public class TSTLookup extends Lookup {
   }
 
   // pre-order traversal
-  private void writeRecursively(DataOutputStream out, TernaryTreeNode node) throws IOException {
+  private void writeRecursively(DataOutput out, TernaryTreeNode node) throws IOException {
     // write out the current node
-    out.writeChar(node.splitchar);
+    out.writeString(new String(new char[] {node.splitchar}, 0, 1));
     // prepare a mask of kids
     byte mask = 0;
     if (node.eqKid != null) mask |= EQ_KID;
@@ -183,7 +182,7 @@ public class TSTLookup extends Lookup {
     if (node.token != null) mask |= HAS_TOKEN;
     if (node.val != null) mask |= HAS_VALUE;
     out.writeByte(mask);
-    if (node.token != null) out.writeUTF(node.token);
+    if (node.token != null) out.writeString(node.token);
     if (node.val != null) out.writeLong(((Number)node.val).longValue());
     // recurse and write kids
     if (node.loKid != null) {
@@ -198,26 +197,15 @@ public class TSTLookup extends Lookup {
   }
 
   @Override
-  public synchronized boolean store(OutputStream output) throws IOException {
-    DataOutputStream out = new DataOutputStream(output);
-    try {
-      writeRecursively(out, root);
-      out.flush();
-    } finally {
-      IOUtils.close(output);
-    }
+  public synchronized boolean store(DataOutput output) throws IOException {
+    writeRecursively(output, root);
     return true;
   }
 
   @Override
-  public synchronized boolean load(InputStream input) throws IOException {
-    DataInputStream in = new DataInputStream(input);
+  public synchronized boolean load(DataInput input) throws IOException {
     root = new TernaryTreeNode();
-    try {
-      readRecursively(in, root);
-    } finally {
-      IOUtils.close(in);
-    }
+    readRecursively(input, root);
     return true;
   }
 
diff --git lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java
index 35ed90c..48bdb7b 100644
--- lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java
+++ lucene/suggest/src/test/org/apache/lucene/search/spell/TestLuceneDictionary.java
@@ -90,7 +90,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
       indexReader = DirectoryReader.open(store);
 
       ld = new LuceneDictionary(indexReader, "nonexistent_field");
-      it = ld.getWordsIterator();
+      it = ld.getEntryIterator();
 
       assertNull("More elements than expected", spare = it.next());
     } finally {
@@ -103,7 +103,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
       indexReader = DirectoryReader.open(store);
 
       ld = new LuceneDictionary(indexReader, "aaa");
-      it = ld.getWordsIterator();
+      it = ld.getEntryIterator();
       assertNotNull("First element doesn't exist.", spare = it.next());
       assertTrue("First element isn't correct", spare.utf8ToString().equals("foo"));
       assertNull("More elements than expected", it.next());
@@ -117,7 +117,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
       indexReader = DirectoryReader.open(store);
 
       ld = new LuceneDictionary(indexReader, "contents");
-      it = ld.getWordsIterator();
+      it = ld.getEntryIterator();
 
       assertNotNull("First element doesn't exist.", spare = it.next());
       assertTrue("First element isn't correct", spare.utf8ToString().equals("Jerry"));
@@ -126,7 +126,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
       assertNull("More elements than expected", it.next());
 
       ld = new LuceneDictionary(indexReader, "contents");
-      it = ld.getWordsIterator();
+      it = ld.getEntryIterator();
 
       int counter = 2;
       while (it.next() != null) {
@@ -145,7 +145,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
       indexReader = DirectoryReader.open(store);
 
       ld = new LuceneDictionary(indexReader, "contents");
-      it = ld.getWordsIterator();
+      it = ld.getEntryIterator();
 
       // just iterate through words
       assertEquals("First element isn't correct", "Jerry", it.next().utf8ToString());
@@ -162,7 +162,7 @@ public class TestLuceneDictionary extends LuceneTestCase {
       indexReader = DirectoryReader.open(store);
 
       ld = new LuceneDictionary(indexReader, "zzz");
-      it = ld.getWordsIterator();
+      it = ld.getEntryIterator();
 
       assertNotNull("First element doesn't exist.", spare = it.next());
       assertEquals("First element isn't correct", "bar", spare.utf8ToString());
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java
index 9e5d8e8..c0a5185 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java
@@ -109,7 +109,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
     Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME, PAYLOAD_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = dictionary.getEntryIterator();
 
     assertNull(inputIterator.next());
     assertEquals(inputIterator.weight(), 0);
@@ -135,7 +135,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
     Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME, PAYLOAD_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
@@ -170,7 +170,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
     Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
@@ -228,7 +228,7 @@ public class DocumentDictionaryTest extends LuceneTestCase {
     IndexReader ir = DirectoryReader.open(dir);
     assertEquals(ir.numDocs(), docs.size());
     Dictionary dictionary = new DocumentDictionary(ir, FIELD_NAME, WEIGHT_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
index b1eef56..d168fc6 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentValueSourceDictionaryTest.java
@@ -83,7 +83,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
     Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
 
     assertNull(inputIterator.next());
     assertEquals(inputIterator.weight(), 0);
@@ -109,7 +109,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IndexReader ir = DirectoryReader.open(dir);
     ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
     Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
@@ -141,7 +141,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     IndexReader ir = DirectoryReader.open(dir);
     ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2), new LongFieldSource(WEIGHT_FIELD_NAME_3)};
     Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new SumFloatFunction(toAdd));
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
@@ -195,7 +195,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
     ValueSource[] toAdd = new ValueSource[] {new LongFieldSource(WEIGHT_FIELD_NAME_1), new LongFieldSource(WEIGHT_FIELD_NAME_2)};
 
     Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME,  new SumFloatFunction(toAdd), PAYLOAD_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
@@ -226,7 +226,7 @@ public class DocumentValueSourceDictionaryTest extends LuceneTestCase {
 
     IndexReader ir = DirectoryReader.open(dir);
     Dictionary dictionary = new DocumentValueSourceDictionary(ir, FIELD_NAME, new DoubleConstValueSource(10), PAYLOAD_FIELD_NAME);
-    InputIterator inputIterator = (InputIterator) dictionary.getWordsIterator();
+    InputIterator inputIterator = (InputIterator) dictionary.getEntryIterator();
     BytesRef f;
     while((f = inputIterator.next())!=null) {
       Document doc = docs.remove(f.utf8ToString());
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/FileDictionaryTest.java lucene/suggest/src/test/org/apache/lucene/search/suggest/FileDictionaryTest.java
index fbeb6df..665af98 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/FileDictionaryTest.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/FileDictionaryTest.java
@@ -76,7 +76,7 @@ public class FileDictionaryTest extends LuceneTestCase {
     InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
     FileDictionary dictionary = new FileDictionary(inputReader);
     List<List<String>> entries = fileInput.getKey();
-    InputIterator inputIter = dictionary.getWordsIterator();
+    InputIterator inputIter = dictionary.getEntryIterator();
     assertFalse(inputIter.hasPayloads());
     BytesRef term;
     int count = 0;
@@ -98,7 +98,7 @@ public class FileDictionaryTest extends LuceneTestCase {
     InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
     FileDictionary dictionary = new FileDictionary(inputReader);
     List<List<String>> entries = fileInput.getKey();
-    InputIterator inputIter = dictionary.getWordsIterator();
+    InputIterator inputIter = dictionary.getEntryIterator();
     assertFalse(inputIter.hasPayloads());
     BytesRef term;
     int count = 0;
@@ -120,7 +120,7 @@ public class FileDictionaryTest extends LuceneTestCase {
     InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
     FileDictionary dictionary = new FileDictionary(inputReader);
     List<List<String>> entries = fileInput.getKey();
-    InputIterator inputIter = dictionary.getWordsIterator();
+    InputIterator inputIter = dictionary.getEntryIterator();
     assertTrue(inputIter.hasPayloads());
     BytesRef term;
     int count = 0;
@@ -146,7 +146,7 @@ public class FileDictionaryTest extends LuceneTestCase {
     InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
     FileDictionary dictionary = new FileDictionary(inputReader);
     List<List<String>> entries = fileInput.getKey();
-    InputIterator inputIter = dictionary.getWordsIterator();
+    InputIterator inputIter = dictionary.getEntryIterator();
     assertTrue(inputIter.hasPayloads());
     BytesRef term;
     int count = 0;
@@ -173,7 +173,7 @@ public class FileDictionaryTest extends LuceneTestCase {
     InputStream inputReader = new ByteArrayInputStream(fileInput.getValue().getBytes("UTF-8"));
     FileDictionary dictionary = new FileDictionary(inputReader, " , ");
     List<List<String>> entries = fileInput.getKey();
-    InputIterator inputIter = dictionary.getWordsIterator();
+    InputIterator inputIter = dictionary.getEntryIterator();
     assertTrue(inputIter.hasPayloads());
     BytesRef term;
     int count = 0;
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java
index 576d2a5..0d2d786 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/TestHighFrequencyDictionary.java
@@ -35,7 +35,7 @@ public class TestHighFrequencyDictionary extends LuceneTestCase {
     writer.close();
     IndexReader ir = DirectoryReader.open(dir);
     Dictionary dictionary = new HighFrequencyDictionary(ir, "bogus", 0.1f);
-    BytesRefIterator tf = dictionary.getWordsIterator();
+    BytesRefIterator tf = dictionary.getEntryIterator();
     assertNull(tf.next());
     dir.close();
   }
diff --git lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java
index e500d34..e9fa29f 100644
--- lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java
+++ lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/TestFreeTextSuggester.java
@@ -171,6 +171,7 @@ public class TestFreeTextSuggester extends LuceneTestCase {
         public boolean hasPayloads() {
           return false;
         }
+
       });
     if (VERBOSE) {
       System.out.println(sug.sizeInBytes() + " bytes");
