You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/08/22 02:07:16 UTC

svn commit: r1619635 - in /lucene/dev/trunk: lucene/ lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/ lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/ solr/core/src/java/org/apache/solr/spelling/suggest/fst/

Author: mikemccand
Date: Fri Aug 22 00:07:15 2014
New Revision: 1619635

URL: http://svn.apache.org/r1619635
Log:
LUCENE-5889: add commit method to AnalyzingInfixSuggester

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
    lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Fri Aug 22 00:07:15 2014
@@ -103,6 +103,9 @@ New Features
 * LUCENE-5895: Lucene now stores a unique id per-segment and per-commit to aid
   in accurate replication of index files (Robert Muir, Mike McCandless)
 
+* LUCENE-5889: Add commit method to AnalyzingInfixSuggester, and allow just using .add
+  to build up the suggester.  (Varun Thacker via Mike McCandless)
+
 ======================= Lucene 4.10.0 ======================
 
 New Features

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java Fri Aug 22 00:07:15 2014
@@ -126,6 +126,7 @@ public class AnalyzingInfixSuggester ext
   final Version matchVersion;
   private final Directory dir;
   final int minPrefixChars;
+  private final boolean commitOnBuild;
 
   /** Used for ongoing NRT additions/updates. */
   private IndexWriter writer;
@@ -146,7 +147,7 @@ public class AnalyzingInfixSuggester ext
    *  Lucene index).  Note that {@link #close}
    *  will also close the provided directory. */
   public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer analyzer) throws IOException {
-    this(matchVersion, dir, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS);
+    this(matchVersion, dir, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS, false);
   }
 
   /** Create a new instance, loading from a previously built
@@ -160,8 +161,12 @@ public class AnalyzingInfixSuggester ext
    *     Prefixes shorter than this are indexed as character
    *     ngrams (increasing index size but making lookups
    *     faster).
+   *
+   *  @param commitOnBuild Call commit after the index has finished building. This would persist the
+   *                       suggester index to disk and future instances of this suggester can use this pre-built dictionary.
    */
-  public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars) throws IOException {
+  public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars,
+                                 boolean commitOnBuild) throws IOException {
 
     if (minPrefixChars < 0) {
       throw new IllegalArgumentException("minPrefixChars must be >= 0; got: " + minPrefixChars);
@@ -172,6 +177,7 @@ public class AnalyzingInfixSuggester ext
     this.matchVersion = matchVersion;
     this.dir = dir;
     this.minPrefixChars = minPrefixChars;
+    this.commitOnBuild = commitOnBuild;
 
     if (DirectoryReader.indexExists(dir)) {
       // Already built; open it:
@@ -237,7 +243,9 @@ public class AnalyzingInfixSuggester ext
       }
 
       //System.out.println("initial indexing time: " + ((System.nanoTime()-t0)/1000000) + " msec");
-
+      if (commitOnBuild) {
+        commit();
+      }
       searcherMgr = new SearcherManager(writer, true, null);
       success = true;
     } finally {
@@ -248,6 +256,13 @@ public class AnalyzingInfixSuggester ext
     }
   }
 
+  public void commit() throws IOException {
+    if (writer == null) {
+      throw new IllegalStateException("Cannot commit on an closed writer. Add documents first");
+    }
+    writer.commit();
+  }
+
   private Analyzer getGramAnalyzer() {
     return new AnalyzerWrapper(Analyzer.PER_FIELD_REUSE_STRATEGY) {
       @Override
@@ -273,12 +288,25 @@ public class AnalyzingInfixSuggester ext
     };
   }
 
+  private synchronized void ensureOpen() throws IOException {
+    if (writer == null) {
+      if (searcherMgr != null) {
+        searcherMgr.close();
+        searcherMgr = null;
+      }
+      writer = new IndexWriter(dir,
+          getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
+      searcherMgr = new SearcherManager(writer, true, null);
+    }
+  }
+
   /** Adds a new suggestion.  Be sure to use {@link #update}
    *  instead if you want to replace a previous suggestion.
    *  After adding or updating a batch of new suggestions,
    *  you must call {@link #refresh} in the end in order to
    *  see the suggestions in {@link #lookup} */
   public void add(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
+    ensureOpen();
     writer.addDocument(buildDocument(text, contexts, weight, payload));
   }
 
@@ -290,6 +318,7 @@ public class AnalyzingInfixSuggester ext
    *  new suggestions, you must call {@link #refresh} in the
    *  end in order to see the suggestions in {@link #lookup} */
   public void update(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
+    ensureOpen();
     writer.updateDocument(new Term(EXACT_TEXT_FIELD_NAME, text.utf8ToString()),
                           buildDocument(text, contexts, weight, payload));
   }
@@ -321,6 +350,9 @@ public class AnalyzingInfixSuggester ext
    *  up" many additions/updates, and then call refresh
    *  once in the end. */
   public void refresh() throws IOException {
+    if (searcherMgr == null) {
+      throw new IllegalStateException("suggester was not built");
+    }
     searcherMgr.maybeRefreshBlocking();
   }
 
@@ -673,6 +705,9 @@ public class AnalyzingInfixSuggester ext
 
   @Override
   public long getCount() throws IOException {
+    if (searcherMgr == null) {
+      return 0;
+    }
     IndexSearcher searcher = searcherMgr.acquire();
     try {
       return searcher.getIndexReader().numDocs();

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java Fri Aug 22 00:07:15 2014
@@ -106,11 +106,13 @@ public class BlendedInfixSuggester exten
    *
    * @param blenderType Type of blending strategy, see BlenderType for more precisions
    * @param numFactor   Factor to multiply the number of searched elements before ponderate
+   * @param commitOnBuild Call commit after the index has finished building. This would persist the
+   *                      suggester index to disk and future instances of this suggester can use this pre-built dictionary.
    * @throws IOException If there are problems opening the underlying Lucene index.
    */
   public BlendedInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer,
-                               int minPrefixChars, BlenderType blenderType, int numFactor) throws IOException {
-    super(matchVersion, dir, indexAnalyzer, queryAnalyzer, minPrefixChars);
+                               int minPrefixChars, BlenderType blenderType, int numFactor, boolean commitOnBuild) throws IOException {
+    super(matchVersion, dir, indexAnalyzer, queryAnalyzer, minPrefixChars, commitOnBuild);
     this.blenderType = blenderType;
     this.numFactor = numFactor;
   }

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java Fri Aug 22 00:07:15 2014
@@ -51,7 +51,7 @@ public class AnalyzingInfixSuggesterTest
     };
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(keys));
 
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
@@ -94,12 +94,12 @@ public class AnalyzingInfixSuggesterTest
     File tempDir = createTempDir("AnalyzingInfixSuggesterTest");
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3, false);
     suggester.build(new InputArrayIterator(keys));
     assertEquals(2, suggester.getCount());
     suggester.close();
 
-    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
+    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3, false);
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
     assertEquals(2, results.size());
     assertEquals("a penny saved is a penny <b>ear</b>ned", results.get(0).key);
@@ -138,7 +138,7 @@ public class AnalyzingInfixSuggesterTest
     };
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3) {
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false) {
         @Override
         protected Object highlight(String text, Set<String> matchedTokens, String prefixToken) throws IOException {
           try (TokenStream ts = queryAnalyzer.tokenStream("text", new StringReader(text))) {
@@ -214,7 +214,7 @@ public class AnalyzingInfixSuggesterTest
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
     int minPrefixLength = random().nextInt(10);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixLength);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixLength, false);
     suggester.build(new InputArrayIterator(keys));
 
     for(int i=0;i<2;i++) {
@@ -271,7 +271,7 @@ public class AnalyzingInfixSuggesterTest
 
       // Make sure things still work after close and reopen:
       suggester.close();
-      suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixLength);
+      suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixLength, false);
     }
     suggester.close();
   }
@@ -282,7 +282,7 @@ public class AnalyzingInfixSuggesterTest
     };
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("penn", random()), 10, true, true);
     assertEquals(1, results.size());
@@ -296,7 +296,7 @@ public class AnalyzingInfixSuggesterTest
     };
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, true);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(keys));
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("penn", random()), 10, true, true);
     assertEquals(1, results.size());
@@ -305,7 +305,7 @@ public class AnalyzingInfixSuggesterTest
 
     // Try again, but overriding addPrefixMatch to highlight
     // the entire hit:
-    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3) {
+    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false) {
         @Override
         protected void addPrefixMatch(StringBuilder sb, String surface, String analyzed, String prefixToken) {
           sb.append("<b>");
@@ -326,7 +326,7 @@ public class AnalyzingInfixSuggesterTest
     };
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(keys));
     suggester.close();
     suggester.close();
@@ -352,7 +352,7 @@ public class AnalyzingInfixSuggesterTest
         }
       };
 
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), indexAnalyzer, queryAnalyzer, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), indexAnalyzer, queryAnalyzer, 3, false);
 
     Input keys[] = new Input[] {
       new Input("a bob for apples", 10, new BytesRef("foobaz")),
@@ -367,7 +367,7 @@ public class AnalyzingInfixSuggesterTest
 
   public void testEmptyAtStart() throws Exception {
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(new Input[0]));
     suggester.add(new BytesRef("a penny saved is a penny earned"), null, 10, new BytesRef("foobaz"));
     suggester.add(new BytesRef("lend me your ear"), null, 8, new BytesRef("foobar"));
@@ -405,7 +405,7 @@ public class AnalyzingInfixSuggesterTest
 
   public void testBothExactAndPrefix() throws Exception {
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(new Input[0]));
     suggester.add(new BytesRef("the pen is pretty"), null, 10, new BytesRef("foobaz"));
     suggester.refresh();
@@ -478,7 +478,7 @@ public class AnalyzingInfixSuggesterTest
       System.out.println("  minPrefixChars=" + minPrefixChars);
     }
 
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixChars);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixChars, false);
 
     // Initial suggester built with nothing:
     suggester.build(new InputArrayIterator(new Input[0]));
@@ -558,7 +558,7 @@ public class AnalyzingInfixSuggesterTest
         }
         lookupThread.finish();
         suggester.close();
-        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixChars);
+        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, minPrefixChars, false);
         lookupThread = new LookupThread(suggester);
         lookupThread.start();
 
@@ -730,7 +730,7 @@ public class AnalyzingInfixSuggesterTest
     };
 
     Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
-    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3);
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newDirectory(), a, a, 3, false);
     suggester.build(new InputArrayIterator(keys));
 
     List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("ear", random()), 10, true, true);
@@ -790,6 +790,65 @@ public class AnalyzingInfixSuggesterTest
     suggester.close();
   }
 
+  public void testNRTWithParallelAdds() throws IOException, InterruptedException {
+    String[] keys = new String[] {"python", "java", "c", "scala", "ruby", "clojure", "erlang", "go", "swift", "lisp"};
+    Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+    File tempDir = createTempDir("AIS_NRT_PERSIST_TEST");
+    AnalyzingInfixSuggester suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3, false);
+    Thread[] multiAddThreads = new Thread[10];
+    try {
+      suggester.refresh();
+      fail("Cannot call refresh on an suggester when no docs are added to the index");
+    } catch(IllegalStateException e) {
+      //Expected
+    }
+    for(int i=0; i<10; i++) {
+      multiAddThreads[i] = new Thread(new IndexDocument(suggester, keys[i]));
+    }
+    for(int i=0; i<10; i++) {
+      multiAddThreads[i].start();
+    }
+    //Make sure all threads have completed indexing
+    for(int i=0; i<10; i++) {
+      multiAddThreads[i].join();
+    }
+
+    suggester.refresh();
+    List<LookupResult> results = suggester.lookup(TestUtil.stringToCharSequence("python", random()), 10, true, false);
+    assertEquals(1, results.size());
+    assertEquals("python", results.get(0).key);
+
+    //Test if the index is getting persisted correctly and can be reopened.
+    suggester.commit();
+    suggester.close();
+
+    suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3, false);
+    results = suggester.lookup(TestUtil.stringToCharSequence("python", random()), 10, true, false);
+    assertEquals(1, results.size());
+    assertEquals("python", results.get(0).key);
+
+    suggester.close();
+  }
+
+  private class IndexDocument implements Runnable {
+    AnalyzingInfixSuggester suggester;
+    String key;
+
+    private IndexDocument(AnalyzingInfixSuggester suggester, String key) {
+      this.suggester = suggester;
+      this.key = key;
+    }
+
+    @Override
+    public void run() {
+      try {
+        suggester.add(new BytesRef(key), null, 10, null);
+      } catch (IOException e) {
+        fail("Could not build suggest dictionary correctly");
+      }
+    }
+  }
+
   private Set<BytesRef> asSet(String... values) {
     HashSet<BytesRef> result = new HashSet<>();
     for(String value : values) {
@@ -812,11 +871,11 @@ public class AnalyzingInfixSuggesterTest
       AnalyzingInfixSuggester suggester;
       Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
       if (iter == 0) {
-        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
+        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3, false);
         suggester.build(new InputArrayIterator(keys));
       } else {
         // Test again, after close/reopen:
-        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3);
+        suggester = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a, 3, false);
       }
 
       // No context provided, all results returned

Modified: lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java Fri Aug 22 00:07:15 2014
@@ -50,7 +50,7 @@ public class BlendedInfixSuggesterTest e
     BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
                                                                 AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
                                                                 BlendedInfixSuggester.BlenderType.POSITION_LINEAR,
-                                                                BlendedInfixSuggester.DEFAULT_NUM_FACTOR);
+                                                                BlendedInfixSuggester.DEFAULT_NUM_FACTOR, false);
     suggester.build(new InputArrayIterator(keys));
 
     // we query for star wars and check that the weight
@@ -98,7 +98,8 @@ public class BlendedInfixSuggesterTest e
 
     // BlenderType.RECIPROCAL is using 1/(1+p) * w where w is weight and p the position of the word
     suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
-                                          AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1);
+                                          AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
+                                          BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1, false);
     suggester.build(new InputArrayIterator(keys));
 
     assertEquals(w, getInResults(suggester, "top", pl, 1));
@@ -129,7 +130,8 @@ public class BlendedInfixSuggesterTest e
 
     // if factor is small, we don't get the expected element
     BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
-                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1);
+                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
+                                                                BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 1, false);
 
     suggester.build(new InputArrayIterator(keys));
 
@@ -148,7 +150,8 @@ public class BlendedInfixSuggesterTest e
 
     // if we increase the factor we have it
     suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
-                                          AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 2);
+                                          AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
+                                          BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL, 2, false);
     suggester.build(new InputArrayIterator(keys));
 
     // we have it
@@ -179,8 +182,9 @@ public class BlendedInfixSuggesterTest e
 
     // if factor is small, we don't get the expected element
     BlendedInfixSuggester suggester = new BlendedInfixSuggester(TEST_VERSION_CURRENT, newFSDirectory(tempDir), a, a,
-                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS, BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL,
-                                                                BlendedInfixSuggester.DEFAULT_NUM_FACTOR);
+                                                                AnalyzingInfixSuggester.DEFAULT_MIN_PREFIX_CHARS,
+                                                                BlendedInfixSuggester.BlenderType.POSITION_RECIPROCAL,
+                                                                BlendedInfixSuggester.DEFAULT_NUM_FACTOR, false);
     suggester.build(new InputArrayIterator(keys));
 
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/AnalyzingInfixLookupFactory.java Fri Aug 22 00:07:15 2014
@@ -94,7 +94,7 @@ public class AnalyzingInfixLookupFactory
     try {
       return new AnalyzingInfixSuggester(core.getSolrConfig().luceneMatchVersion, 
                                          FSDirectory.open(new File(indexPath)), indexAnalyzer,
-                                         queryAnalyzer, minPrefixChars);
+                                         queryAnalyzer, minPrefixChars, true);
     } catch (IOException e) {
       throw new RuntimeException();
     }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java?rev=1619635&r1=1619634&r2=1619635&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/suggest/fst/BlendedInfixLookupFactory.java Fri Aug 22 00:07:15 2014
@@ -100,7 +100,7 @@ public class BlendedInfixLookupFactory e
       return new BlendedInfixSuggester(core.getSolrConfig().luceneMatchVersion, 
                                        FSDirectory.open(new File(indexPath)),
                                        indexAnalyzer, queryAnalyzer, minPrefixChars,
-                                       blenderType, numFactor);
+                                       blenderType, numFactor, true);
     } catch (IOException e) {
       throw new RuntimeException();
     }