diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
index 1647571..2182562 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
@@ -23,6 +23,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.StringReader;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
@@ -530,6 +531,9 @@ public class AnalyzingSuggester extends Lookup {
   public boolean store(OutputStream output) throws IOException {
     DataOutput dataOut = new OutputStreamDataOutput(output);
     try {
+      if (fst == null)
+        return false;
+
       fst.save(dataOut);
       dataOut.writeVInt(maxAnalyzedPathsForOneInput);
     } finally {
@@ -557,6 +561,8 @@ public class AnalyzingSuggester extends Lookup {
     if (onlyMorePopular) {
       throw new IllegalArgumentException("this suggester only works with onlyMorePopular=false");
     }
+    if (fst == null)
+      return Collections.emptyList();
 
     //System.out.println("lookup key=" + key + " num=" + num);
     final BytesRef utf8Key = new BytesRef(key);
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
index 2c93863..6d7f559 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
@@ -147,6 +147,9 @@ public class WFSTCompletionLookup extends Lookup {
       throw new IllegalArgumentException("this suggester only works with onlyMorePopular=false");
     }
 
+    if (fst == null)
+      return Collections.emptyList();
+
     BytesRef scratch = new BytesRef(key);
     int prefixLength = scratch.length;
     Arc<Long> arc = new Arc<Long>();
@@ -219,6 +222,8 @@ public class WFSTCompletionLookup extends Lookup {
    * or null if it does not exist.
    */
   public Object get(CharSequence key) {
+    if (fst == null)
+      return null;
     Arc<Long> arc = new Arc<Long>();
     Long result = null;
     try {
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
index 2ac25c7..0057d9a 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
@@ -134,6 +134,15 @@ public class AnalyzingSuggesterTest extends LuceneTestCase {
     assertEquals(50, results.get(0).value, 0.01F);
   }
 
+  public void testEmpty() throws Exception {
+    Analyzer standard = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true, MockTokenFilter.ENGLISH_STOPSET, false);
+    AnalyzingSuggester suggester = new AnalyzingSuggester(standard);
+    suggester.build(new TermFreqArrayIterator(new TermFreq[0]));
+
+    List<LookupResult> result = suggester.lookup("a", false, 20);
+    assertTrue(result.isEmpty());
+  }
+
   public void testNoSeps() throws Exception {
     TermFreq[] keys = new TermFreq[] {
       new TermFreq("ab cd", 0),
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
index f7398a8..b473e4d 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
@@ -263,8 +263,14 @@ public class FuzzySuggesterTest extends LuceneTestCase {
     assertEquals("wi fi network is fast", results.get(1).key);
     assertEquals(10, results.get(1).value);
   }
-  
-  
+
+  public void testEmpty() throws Exception {
+    FuzzySuggester suggester = new FuzzySuggester(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
+    suggester.build(new TermFreqArrayIterator(new TermFreq[0]));
+
+    List<LookupResult> result = suggester.lookup("a", false, 20);
+    assertTrue(result.isEmpty());
+  }
 
   public void testInputPathRequired() throws Exception {
 
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
index d50e710..b2c6e44 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/WFSTCompletionTest.java
@@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest.fst;
 
 import java.util.*;
 
+import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.TermFreq;
 import org.apache.lucene.search.suggest.TermFreqArrayIterator;
@@ -209,4 +210,12 @@ public class WFSTCompletionTest extends LuceneTestCase {
           new TermFreq(key2, 50),
         }));
   }
+
+  public void testEmpty() throws Exception {
+    WFSTCompletionLookup suggester = new WFSTCompletionLookup(false);
+
+    suggester.build(new TermFreqArrayIterator(new TermFreq[0]));
+    List<LookupResult> result = suggester.lookup("a", false, 20);
+    assertTrue(result.isEmpty());
+  }
 }
