You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jd...@apache.org on 2012/06/04 19:55:26 UTC

svn commit: r1346058 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/spelling/ core/src/test-files/solr/conf/ core/src/test/org/apache/solr/handler/component/ core/src/test/org/apache...

Author: jdyer
Date: Mon Jun  4 17:55:25 2012
New Revision: 1346058

URL: http://svn.apache.org/viewvc?rev=1346058&view=rev
Log:
SOLR-2993:  o.a.l.Search.Suggest.WordBreakSpellChecker Solr integration

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ResultEntry.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/WordBreakSolrSpellChecker.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/WordBreakSolrSpellCheckerTest.java   (with props)
Removed:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/RankedSpellPossibility.java
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/QueryConverter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java
    lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-spellcheckcomponent.xml
    lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig.xml
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellPossibilityIteratorTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellingQueryConverterTest.java
    lucene/dev/trunk/solr/example/solr/conf/solrconfig.xml

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Jun  4 17:55:25 2012
@@ -337,6 +337,12 @@ New Features
     UUIDUpdateProcessorFactory
     DefaultValueUpdateProcessorFactory
   (hossman)
+  
+* SOLR-2993: Add WordBreakSolrSpellChecker to offer suggestions by combining adjacent 
+  query terms and/or breaking terms into multiple words.  This spellchecker can be 
+  configured with a traditional checker (ie: DirectSolrSpellChecker).  The results 
+  are combined and collations can contain a mix of corrections from both spellcheckers.  
+  (James Dyer)
 
 Optimizations
 ----------------------

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Mon Jun  4 17:55:25 2012
@@ -124,6 +124,7 @@ public class SpellCheckComponent extends
     String q = params.get(SPELLCHECK_Q);
     SolrSpellChecker spellChecker = getSpellChecker(params);
     Collection<Token> tokens = null;
+    
     if (q != null) {
       //we have a spell check param, tokenize it with the query analyzer applicable for this spellchecker
       tokens = getTokens(q, spellChecker.getQueryAnalyzer());
@@ -143,7 +144,10 @@ public class SpellCheckComponent extends
         float accuracy = params.getFloat(SPELLCHECK_ACCURACY, Float.MIN_VALUE);
         Integer alternativeTermCount = params.getInt(SpellingParams.SPELLCHECK_ALTERNATIVE_TERM_COUNT); 
         Integer maxResultsForSuggest = params.getInt(SpellingParams.SPELLCHECK_MAX_RESULTS_FOR_SUGGEST);
-        SolrParams customParams = getCustomParams(getDictionaryName(params), params);
+        ModifiableSolrParams customParams = new ModifiableSolrParams();
+        for (String checkerName : getDictionaryNames(params)) {
+          customParams.add(getCustomParams(checkerName, params));
+        }
         
         Integer hitsInteger = (Integer) rb.rsp.getToLog().get("hits");
         long hits = 0;
@@ -173,7 +177,7 @@ public class SpellCheckComponent extends
         NamedList suggestions = toNamedList(shardRequest, spellingResult, q,
             extendedResults, collate, isCorrectlySpelled);
         if (collate) {
-          addCollationsToResponse(params, spellingResult, rb, q, suggestions);
+          addCollationsToResponse(params, spellingResult, rb, q, suggestions, spellChecker.isSuggestionsMayOverlap());
         }
         NamedList response = new SimpleOrderedMap();
         response.add("suggestions", suggestions);
@@ -181,14 +185,14 @@ public class SpellCheckComponent extends
 
       } else {
         throw new SolrException(SolrException.ErrorCode.NOT_FOUND,
-            "Specified dictionary does not exist: " + getDictionaryName(params));
+            "Specified dictionaries do not exist: " + getDictionaryNameAsSingleString(getDictionaryNames(params)));
       }
     }
   }
   
   @SuppressWarnings("unchecked")
 	protected void addCollationsToResponse(SolrParams params, SpellingResult spellingResult, ResponseBuilder rb, String q,
-			NamedList response) {
+	    NamedList response, boolean suggestionsMayOverlap) {
 		int maxCollations = params.getInt(SPELLCHECK_MAX_COLLATIONS, 1);
 		int maxCollationTries = params.getInt(SPELLCHECK_MAX_COLLATION_TRIES, 0);
 		int maxCollationEvaluations = params.getInt(SPELLCHECK_MAX_COLLATION_EVALUATIONS, 10000);
@@ -196,8 +200,8 @@ public class SpellCheckComponent extends
 		boolean shard = params.getBool(ShardParams.IS_SHARD, false);
 
 		SpellCheckCollator collator = new SpellCheckCollator();
-		List<SpellCheckCollation> collations = collator.collate(spellingResult, q, rb, maxCollations, maxCollationTries, maxCollationEvaluations);
-		//by sorting here we guarantee a non-distributed request returns all 
+		List<SpellCheckCollation> collations = collator.collate(spellingResult, q, rb, maxCollations, maxCollationTries, maxCollationEvaluations, suggestionsMayOverlap);
+    //by sorting here we guarantee a non-distributed request returns all 
 		//results in the same order as a distributed request would, 
 		//even in cases when the internal rank is the same.
 		Collections.sort(collations);
@@ -459,13 +463,38 @@ public class SpellCheckComponent extends
   }
 
   protected SolrSpellChecker getSpellChecker(SolrParams params) {
-    return spellCheckers.get(getDictionaryName(params));
+    String[] dictName = getDictionaryNames(params);
+    if (dictName.length == 1) {
+      return spellCheckers.get(dictName[0]);
+    } else {
+      String singleStr = getDictionaryNameAsSingleString(dictName);
+      SolrSpellChecker ssc = spellCheckers.get(singleStr);
+      if (ssc == null) {
+        ConjunctionSolrSpellChecker cssc = new ConjunctionSolrSpellChecker();
+        for (String dn : dictName) {
+          cssc.addChecker(spellCheckers.get(dn));
+        }
+        ssc = cssc;
+      }
+      return ssc;
+    }
+  }
+  
+  private String getDictionaryNameAsSingleString(String[] dictName) {
+    StringBuilder sb = new StringBuilder();
+    for (String dn : dictName) {
+      if (sb.length() > 0) {
+        sb.append(" ");
+      }
+      sb.append(dn);
+    }
+    return sb.toString();
   }
 
-  private String getDictionaryName(SolrParams params) {
-    String dictName = params.get(SPELLCHECK_DICT);
+  private String[] getDictionaryNames(SolrParams params) {
+    String[] dictName = params.getParams(SPELLCHECK_DICT);
     if (dictName == null) {
-      dictName = SolrSpellChecker.DEFAULT_DICTIONARY_NAME;
+      return new String[] {SolrSpellChecker.DEFAULT_DICTIONARY_NAME};
     }
     return dictName;
   }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java?rev=1346058&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java Mon Jun  4 17:55:25 2012
@@ -0,0 +1,215 @@
+package org.apache.solr.spelling;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.search.spell.StringDistance;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.SpellCheckMergeData;
+import org.apache.solr.search.SolrIndexSearcher;
+
+/**
+ * <p>This class lets a query be run through multiple spell checkers.
+ *    The initial use-case is to use {@link WordBreakSolrSpellChecker}
+ *    in conjunction with a "standard" spell checker 
+ *    (such as {@link DirectSolrSpellChecker}
+ *  </p>
+ */
+public class ConjunctionSolrSpellChecker extends SolrSpellChecker {
+  private StringDistance stringDistance = null;
+  private Float accuracy = null;
+  private String dictionaryName = null;
+  private Analyzer queryAnalyzer = null;
+  private List<SolrSpellChecker> checkers = new ArrayList<SolrSpellChecker>();
+  private boolean initalized = false;
+  
+  public void addChecker(SolrSpellChecker checker) {
+    if (initalized) {
+      throw new IllegalStateException(
+          "Need to add checkers before calling init()");
+    }
+    try {
+      if (stringDistance == null) {
+        stringDistance = checker.getStringDistance();
+      } else if (stringDistance != checker.getStringDistance()) {
+        throw new IllegalArgumentException(
+            "All checkers need to use the same StringDistance.");
+      }
+    } catch (UnsupportedOperationException uoe) {
+      // ignore
+    }
+    try {
+      if (accuracy == null) {
+        accuracy = checker.getAccuracy();
+      } else if (accuracy != checker.getAccuracy()) {
+        throw new IllegalArgumentException(
+            "All checkers need to use the same Accuracy.");
+      }
+    } catch (UnsupportedOperationException uoe) {
+      // ignore
+    }
+    if (queryAnalyzer == null) {
+      queryAnalyzer = checker.getQueryAnalyzer();
+    } else if (queryAnalyzer != checker.getQueryAnalyzer()) {
+      throw new IllegalArgumentException(
+          "All checkers need to use the same Analyzer.");
+    }
+    checkers.add(checker);
+  }
+  
+  @SuppressWarnings("unchecked")
+  @Override
+  public String init(NamedList config, SolrCore core) {
+    for (int i = 0; i < checkers.size(); i++) {
+      SolrSpellChecker c = checkers.get(i);
+      String dn = c.init(config, core);
+      
+      //TODO:  in the future, we could develop this further to allow
+      //        multiple spellcheckers with per-field dictionaries...
+      if (dictionaryName != null && !dictionaryName.equals(dn)) {
+        throw new IllegalArgumentException(
+            "Cannot have more than one dictionary. (" + dn + " , "
+                + dictionaryName + ")");
+      }
+      dictionaryName = dn;
+    }
+    if (dictionaryName == null) {
+      dictionaryName = DEFAULT_DICTIONARY_NAME;
+    }
+    initalized = true;
+    return dictionaryName;
+  }
+  
+  @Override
+  public void build(SolrCore core, SolrIndexSearcher searcher) {
+    for (SolrSpellChecker c : checkers) {
+      c.build(core, searcher);
+    }
+  }
+  
+  @Override
+  public SpellingResult getSuggestions(SpellingOptions options)
+      throws IOException {
+    SpellingResult[] results = new SpellingResult[checkers.size()];
+    for (int i = 0; i < checkers.size(); i++) {
+      results[i] = checkers.get(i).getSuggestions(options);
+    }
+    return mergeCheckers(results, options.count);
+  }
+  
+  @Override
+  public SpellingResult mergeSuggestions(SpellCheckMergeData mergeData,
+      int numSug, int count, boolean extendedResults) {
+    SpellingResult[] results = new SpellingResult[checkers.size()];
+    for (int i = 0; i < checkers.size(); i++) {
+      results[i] = checkers.get(i).mergeSuggestions(mergeData, numSug, count,
+          extendedResults);
+    }
+    return mergeCheckers(results, numSug);
+  }
+  
+  //TODO: This just interleaves the results.  In the future, we might want to let users give each checker its
+  //      own weight and use that in combination to score & frequency to sort the results ?
+  private SpellingResult mergeCheckers(SpellingResult[] results, int numSug) {
+    Map<Token, List<LinkedHashMap<String, Integer>>> allSuggestions = new LinkedHashMap<Token, List<LinkedHashMap<String, Integer>>>();
+    for(SpellingResult result : results) {
+      for(Map.Entry<Token, LinkedHashMap<String, Integer>> entry : result.getSuggestions().entrySet()) {
+        List<LinkedHashMap<String, Integer>> allForThisToken = allSuggestions.get(entry.getKey());
+        if(allForThisToken==null) {
+          allForThisToken = new ArrayList<LinkedHashMap<String, Integer>>();
+          allSuggestions.put(entry.getKey(), allForThisToken);
+        }
+        allForThisToken.add(entry.getValue());
+      }
+    }    
+    SpellingResult combinedResult = new SpellingResult();    
+    for(Map.Entry<Token, List<LinkedHashMap<String, Integer>>> entry : allSuggestions.entrySet()) {
+      Token original = entry.getKey();      
+      List<Iterator<Map.Entry<String,Integer>>> corrIters = new ArrayList<Iterator<Map.Entry<String,Integer>>>(entry.getValue().size());
+      for(LinkedHashMap<String, Integer> corrections : entry.getValue()) {
+        corrIters.add(corrections.entrySet().iterator());
+      }        
+      int numberAdded = 0;
+      while(numberAdded < numSug) {
+        boolean anyData = false;
+        for(Iterator<Map.Entry<String,Integer>> iter : corrIters) {
+          if(iter.hasNext()) {
+            anyData = true;
+            Map.Entry<String,Integer> corr = iter.next();
+            combinedResult.add(original, corr.getKey(), corr.getValue());
+            if(++numberAdded==numSug) {
+              break;
+            }
+          }
+        }        
+        if(!anyData) {
+          break;
+        }
+      }      
+    }    
+    return combinedResult;
+  }
+  
+  @Override
+  public void reload(SolrCore core, SolrIndexSearcher searcher)
+      throws IOException {
+    for (SolrSpellChecker c : checkers) {
+      c.reload(core, searcher);
+    }
+  }
+  
+  @Override
+  public Analyzer getQueryAnalyzer() {
+    return queryAnalyzer;
+  }
+  
+  @Override
+  public String getDictionaryName() {
+    return dictionaryName;
+  }
+  
+  @Override
+  protected float getAccuracy() {
+    if (accuracy == null) {
+      return super.getAccuracy();
+    }
+    return accuracy;
+  }
+  
+  @Override
+  protected StringDistance getStringDistance() {
+    if (stringDistance == null) {
+      return super.getStringDistance();
+    }
+    return stringDistance;
+  }
+  
+  @Override
+  public boolean isSuggestionsMayOverlap() {
+    return true;
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java Mon Jun  4 17:55:25 2012
@@ -1,4 +1,5 @@
 package org.apache.solr.spelling;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -18,12 +19,16 @@ package org.apache.solr.spelling;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.PriorityQueue;
+import java.util.Set;
 
 import org.apache.lucene.analysis.Token;
 
@@ -37,161 +42,388 @@ import org.apache.lucene.analysis.Token;
  * </p>
  * 
  */
-public class PossibilityIterator implements Iterator<RankedSpellPossibility> {
-	private List<List<SpellCheckCorrection>> possibilityList = new ArrayList<List<SpellCheckCorrection>>();
-	private Iterator<RankedSpellPossibility> rankedPossibilityIterator = null;
-	private int correctionIndex[];
-	private boolean done = false;
-
-	@SuppressWarnings("unused")
-	private PossibilityIterator() {
-		throw new AssertionError("You shan't go here.");
-	}
-
-	/**
-	 * <p>
-	 * We assume here that the passed-in inner LinkedHashMaps are already sorted
-	 * in order of "Best Possible Correction".
-	 * </p>
-	 * 
-	 * @param suggestions
-	 */
-	public PossibilityIterator(Map<Token, LinkedHashMap<String, Integer>> suggestions, int maximumRequiredSuggestions, int maxEvaluations) {
-		for (Map.Entry<Token, LinkedHashMap<String, Integer>> entry : suggestions.entrySet()) {
-			Token token = entry.getKey();
-			if(entry.getValue().size()==0) {
-			  continue;
-			}
-			List<SpellCheckCorrection> possibleCorrections = new ArrayList<SpellCheckCorrection>();
-			for (Map.Entry<String, Integer> entry1 : entry.getValue().entrySet()) {
-				SpellCheckCorrection correction = new SpellCheckCorrection();
-				correction.setOriginal(token);
-				correction.setCorrection(entry1.getKey());
-				correction.setNumberOfOccurences(entry1.getValue());
-				possibleCorrections.add(correction);
-			}
-			possibilityList.add(possibleCorrections);
-		}
-
-		int wrapSize = possibilityList.size();
-		if (wrapSize == 0) {
-			done = true;
-		} else {
-			correctionIndex = new int[wrapSize];
-			for (int i = 0; i < wrapSize; i++) {
-				int suggestSize = possibilityList.get(i).size();
-				if (suggestSize == 0) {
-					done = true;
-					break;
-				}
-				correctionIndex[i] = 0;
-			}
-		}
-		
-		long count = 0;
-		PriorityQueue<RankedSpellPossibility> rankedPossibilities = new PriorityQueue<RankedSpellPossibility>();		
-		while (count < maxEvaluations && internalHasNext()) {
-			RankedSpellPossibility rsp = internalNext();
-			count++;			
-			
-			if(rankedPossibilities.size() >= maximumRequiredSuggestions && rsp.getRank() >= rankedPossibilities.peek().getRank()) {
-				continue;
-			}
+public class PossibilityIterator implements
+    Iterator<PossibilityIterator.RankedSpellPossibility> {
+  private List<List<SpellCheckCorrection>> possibilityList = new ArrayList<List<SpellCheckCorrection>>();
+  private Iterator<RankedSpellPossibility> rankedPossibilityIterator = null;
+  private int correctionIndex[];
+  private boolean done = false;
+  private Iterator<List<SpellCheckCorrection>> nextOnes = null;
+  private int nextOnesRank = 0;
+  private int nextOnesIndex = 0;
+  private boolean suggestionsMayOverlap = false;
+  
+  @SuppressWarnings("unused")
+  private PossibilityIterator() {
+    throw new AssertionError("You shan't go here.");
+  }
+  
+  /**
+   * <p>
+   * We assume here that the passed-in inner LinkedHashMaps are already sorted
+   * in order of "Best Possible Correction".
+   * </p>
+   * 
+   * @param suggestions
+   */
+  public PossibilityIterator(
+      Map<Token,LinkedHashMap<String,Integer>> suggestions,
+      int maximumRequiredSuggestions, int maxEvaluations, boolean overlap) {
+    this.suggestionsMayOverlap = overlap;
+    for (Map.Entry<Token,LinkedHashMap<String,Integer>> entry : suggestions
+        .entrySet()) {
+      Token token = entry.getKey();
+      if (entry.getValue().size() == 0) {
+        continue;
+      }
+      List<SpellCheckCorrection> possibleCorrections = new ArrayList<SpellCheckCorrection>();
+      for (Map.Entry<String,Integer> entry1 : entry.getValue().entrySet()) {
+        SpellCheckCorrection correction = new SpellCheckCorrection();
+        correction.setOriginal(token);
+        correction.setCorrection(entry1.getKey());
+        correction.setNumberOfOccurences(entry1.getValue());
+        possibleCorrections.add(correction);
+      }
+      possibilityList.add(possibleCorrections);
+    }
+    
+    int wrapSize = possibilityList.size();
+    if (wrapSize == 0) {
+      done = true;
+    } else {
+      correctionIndex = new int[wrapSize];
+      for (int i = 0; i < wrapSize; i++) {
+        int suggestSize = possibilityList.get(i).size();
+        if (suggestSize == 0) {
+          done = true;
+          break;
+        }
+        correctionIndex[i] = 0;
+      }
+    }
+    PriorityQueue<RankedSpellPossibility> rankedPossibilities = new PriorityQueue<RankedSpellPossibility>(
+        11, new RankComparator());
+    Set<RankedSpellPossibility> removeDuplicates = null;
+    if (suggestionsMayOverlap) {
+      removeDuplicates = new HashSet<RankedSpellPossibility>();
+    }
+    long numEvaluations = 0;
+    while (numEvaluations < maxEvaluations && internalHasNext()) {
+      RankedSpellPossibility rsp = internalNext();
+      numEvaluations++;
+      if (rankedPossibilities.size() >= maximumRequiredSuggestions
+          && rsp.rank >= rankedPossibilities.peek().rank) {
+        continue;
+      }
       if (!isSuggestionForReal(rsp)) {
         continue;
       }
-			rankedPossibilities.offer(rsp);
-			if(rankedPossibilities.size() > maximumRequiredSuggestions) {
-				rankedPossibilities.poll();
-			}
-		}
-		
-		RankedSpellPossibility[] rpArr = new RankedSpellPossibility[rankedPossibilities.size()];
-		for(int i=rankedPossibilities.size() - 1  ; i>=0 ; i--) {
-			rpArr[i] = rankedPossibilities.remove();
-		}
-		rankedPossibilityIterator = Arrays.asList(rpArr).iterator();		
-	}
-	
+      if (removeDuplicates == null) {
+        rankedPossibilities.offer(rsp);
+      } else {
+        // Needs to be in token-offset order so that the match-and-replace
+        // option for collations can work.
+        Collections.sort(rsp.corrections, new StartOffsetComparator());
+        if (removeDuplicates.add(rsp)) {
+          rankedPossibilities.offer(rsp);
+        }
+      }
+      if (rankedPossibilities.size() > maximumRequiredSuggestions) {
+        RankedSpellPossibility removed = rankedPossibilities.poll();
+        if (removeDuplicates != null) {
+          removeDuplicates.remove(removed);
+        }
+      }
+    }
+    
+    RankedSpellPossibility[] rpArr = new RankedSpellPossibility[rankedPossibilities
+        .size()];
+    for (int i = rankedPossibilities.size() - 1; i >= 0; i--) {
+      rpArr[i] = rankedPossibilities.remove();
+    }
+    rankedPossibilityIterator = Arrays.asList(rpArr).iterator();
+  }
+  
   private boolean isSuggestionForReal(RankedSpellPossibility rsp) {
-    for (SpellCheckCorrection corr : rsp.getCorrections()) {
+    for (SpellCheckCorrection corr : rsp.corrections) {
       if (!corr.getOriginalAsString().equals(corr.getCorrection())) {
         return true;
       }
     }
     return false;
   }
-
-	private boolean internalHasNext() {
-		return !done;
-	}
-
-	/**
-	 * <p>
-	 * This method is converting the independent LinkHashMaps containing various
-	 * (silo'ed) suggestions for each mis-spelled word into individual
-	 * "holistic query corrections", aka. "Spell Check Possibility"
-	 * </p>
-	 * <p>
-	 * Rank here is the sum of each selected term's position in its respective
-	 * LinkedHashMap.
-	 * </p>
-	 * 
-	 * @return
-	 */
-	private RankedSpellPossibility internalNext() {
-		if (done) {
-			throw new NoSuchElementException();
-		}
-
-		List<SpellCheckCorrection> possibleCorrection = new ArrayList<SpellCheckCorrection>();
-		int rank = 0;
-		for (int i = 0; i < correctionIndex.length; i++) {
-			List<SpellCheckCorrection> singleWordPossibilities = possibilityList.get(i);
-			SpellCheckCorrection singleWordPossibility = singleWordPossibilities.get(correctionIndex[i]);
-			rank += correctionIndex[i];
-
-			if (i == correctionIndex.length - 1) {
-				correctionIndex[i]++;
-				if (correctionIndex[i] == singleWordPossibilities.size()) {
-					correctionIndex[i] = 0;
-					if (correctionIndex.length == 1) {
-						done = true;
-					}
-					for (int ii = i - 1; ii >= 0; ii--) {
-						correctionIndex[ii]++;
-						if (correctionIndex[ii] >= possibilityList.get(ii).size() && ii > 0) {
-							correctionIndex[ii] = 0;
-						} else {
-							break;
-						}
-					}
-				}
-			}
-			possibleCorrection.add(singleWordPossibility);
-		}
-		
-		if(correctionIndex[0] == possibilityList.get(0).size())
-		{
-			done = true;
-		}
-
-		RankedSpellPossibility rsl = new RankedSpellPossibility();
-		rsl.setCorrections(possibleCorrection);
-		rsl.setRank(rank);
-		return rsl;
-	}
-
-	public boolean hasNext() {
-		return rankedPossibilityIterator.hasNext();
-	}
-
-	public RankedSpellPossibility next() {
-		return rankedPossibilityIterator.next();
-	}
-
-	public void remove() {
-		throw new UnsupportedOperationException();
-	}
-
+  
+  private boolean internalHasNext() {
+    if (nextOnes != null && nextOnes.hasNext()) {
+      return true;
+    }
+    if (done) {
+      return false;
+    }
+    internalNextAdvance();
+    if (nextOnes != null && nextOnes.hasNext()) {
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * <p>
+   * This method is converting the independent LinkHashMaps containing various
+   * (silo'ed) suggestions for each mis-spelled word into individual
+   * "holistic query corrections", aka. "Spell Check Possibility"
+   * </p>
+   * <p>
+   * Rank here is the sum of each selected term's position in its respective
+   * LinkedHashMap.
+   * </p>
+   * 
+   * @return
+   */
+  private RankedSpellPossibility internalNext() {
+    if (nextOnes != null && nextOnes.hasNext()) {
+      RankedSpellPossibility rsl = new RankedSpellPossibility();
+      rsl.corrections = nextOnes.next();
+      rsl.rank = nextOnesRank;
+      rsl.index = nextOnesIndex++;
+      return rsl;
+    }
+    if (done) {
+      throw new NoSuchElementException();
+    }
+    internalNextAdvance();
+    if (nextOnes != null && nextOnes.hasNext()) {
+      RankedSpellPossibility rsl = new RankedSpellPossibility();
+      rsl.corrections = nextOnes.next();
+      rsl.rank = nextOnesRank;
+      rsl.index = nextOnesIndex++;
+      return rsl;
+    }
+    throw new NoSuchElementException();
+  }
+  
+  private void internalNextAdvance() {
+    List<SpellCheckCorrection> possibleCorrection = null;
+    if (nextOnes != null && nextOnes.hasNext()) {
+      possibleCorrection = nextOnes.next();
+    } else {
+      if (done) {
+        throw new NoSuchElementException();
+      }
+      possibleCorrection = new ArrayList<SpellCheckCorrection>();
+      List<List<SpellCheckCorrection>> possibleCorrections = null;
+      int rank = 0;
+      while (!done
+          && (possibleCorrections == null || possibleCorrections.size() == 0)) {
+        rank = 0;
+        for (int i = 0; i < correctionIndex.length; i++) {
+          List<SpellCheckCorrection> singleWordPossibilities = possibilityList
+              .get(i);
+          SpellCheckCorrection singleWordPossibility = singleWordPossibilities
+              .get(correctionIndex[i]);
+          rank += correctionIndex[i];
+          if (i == correctionIndex.length - 1) {
+            correctionIndex[i]++;
+            if (correctionIndex[i] == singleWordPossibilities.size()) {
+              correctionIndex[i] = 0;
+              if (correctionIndex.length == 1) {
+                done = true;
+              }
+              for (int ii = i - 1; ii >= 0; ii--) {
+                correctionIndex[ii]++;
+                if (correctionIndex[ii] >= possibilityList.get(ii).size()
+                    && ii > 0) {
+                  correctionIndex[ii] = 0;
+                } else {
+                  break;
+                }
+              }
+            }
+          }
+          possibleCorrection.add(singleWordPossibility);
+        }
+        if (correctionIndex[0] == possibilityList.get(0).size()) {
+          done = true;
+        }
+        if (suggestionsMayOverlap) {
+          possibleCorrections = separateOverlappingTokens(possibleCorrection);
+        } else {
+          possibleCorrections = new ArrayList<List<SpellCheckCorrection>>(1);
+          possibleCorrections.add(possibleCorrection);
+        }
+      }
+      nextOnes = possibleCorrections.iterator();
+      nextOnesRank = rank;
+      nextOnesIndex = 0;
+    }
+  }
+  
+  private List<List<SpellCheckCorrection>> separateOverlappingTokens(
+      List<SpellCheckCorrection> possibleCorrection) {
+    List<List<SpellCheckCorrection>> ret = null;
+    if (possibleCorrection.size() == 1) {
+      ret = new ArrayList<List<SpellCheckCorrection>>(1);
+      ret.add(possibleCorrection);
+      return ret;
+    }
+    ret = new ArrayList<List<SpellCheckCorrection>>();
+    for (int i = 0; i < possibleCorrection.size(); i++) {
+      List<SpellCheckCorrection> c = compatible(possibleCorrection, i);
+      ret.add(c);
+    }
+    return ret;
+  }
+  
+  private List<SpellCheckCorrection> compatible(List<SpellCheckCorrection> all,
+      int pos) {
+    List<SpellCheckCorrection> priorPassCompatibles = null;
+    {
+      List<SpellCheckCorrection> firstPassCompatibles = new ArrayList<SpellCheckCorrection>(
+          all.size());
+      SpellCheckCorrection sacred = all.get(pos);
+      firstPassCompatibles.add(sacred);
+      int index = pos;
+      boolean gotOne = false;
+      for (int i = 0; i < all.size() - 1; i++) {
+        index++;
+        if (index == all.size()) {
+          index = 0;
+        }
+        SpellCheckCorrection disposable = all.get(index);
+        if (!conflicts(sacred, disposable)) {
+          firstPassCompatibles.add(disposable);
+          gotOne = true;
+        }
+      }
+      if (!gotOne) {
+        return firstPassCompatibles;
+      }
+      priorPassCompatibles = firstPassCompatibles;
+    }
+    
+    {
+      pos = 1;
+      while (true) {
+        if (pos == priorPassCompatibles.size() - 1) {
+          return priorPassCompatibles;
+        }
+        List<SpellCheckCorrection> subsequentPassCompatibles = new ArrayList<SpellCheckCorrection>(
+            priorPassCompatibles.size());
+        SpellCheckCorrection sacred = null;
+        for (int i = 0; i <= pos; i++) {
+          sacred = priorPassCompatibles.get(i);
+          subsequentPassCompatibles.add(sacred);
+        }
+        int index = pos;
+        boolean gotOne = false;
+        for (int i = 0; i < priorPassCompatibles.size() - 1; i++) {
+          index++;
+          if (index == priorPassCompatibles.size()) {
+            break;
+          }
+          SpellCheckCorrection disposable = priorPassCompatibles.get(index);
+          if (!conflicts(sacred, disposable)) {
+            subsequentPassCompatibles.add(disposable);
+            gotOne = true;
+          }
+        }
+        if (!gotOne || pos == priorPassCompatibles.size() - 1) {
+          return subsequentPassCompatibles;
+        }
+        priorPassCompatibles = subsequentPassCompatibles;
+        pos++;
+      }
+    }
+  }
+  
+  private boolean conflicts(SpellCheckCorrection c1, SpellCheckCorrection c2) {
+    int s1 = c1.getOriginal().startOffset();
+    int e1 = c1.getOriginal().endOffset();
+    int s2 = c2.getOriginal().startOffset();
+    int e2 = c2.getOriginal().endOffset();
+    if (s2 >= s1 && s2 <= e1) {
+      return true;
+    }
+    if (s1 >= s2 && s1 <= e2) {
+      return true;
+    }
+    return false;
+  }
+  
+  public boolean hasNext() {
+    return rankedPossibilityIterator.hasNext();
+  }
+  
+  public PossibilityIterator.RankedSpellPossibility next() {
+    return rankedPossibilityIterator.next();
+  }
+  
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+  
+  public class RankedSpellPossibility {
+    public List<SpellCheckCorrection> corrections;
+    public int rank;
+    public int index;
+    
+    @Override
+    // hashCode() and equals() only consider the actual correction, not the rank
+    // or index.
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result
+          + ((corrections == null) ? 0 : corrections.hashCode());
+      return result;
+    }
+    
+    @Override
+    // hashCode() and equals() only consider the actual correction, not the rank
+    // or index.
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      RankedSpellPossibility other = (RankedSpellPossibility) obj;
+      if (corrections == null) {
+        if (other.corrections != null) return false;
+      } else if (!corrections.equals(other.corrections)) return false;
+      return true;
+    }
+    
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("rank=").append(rank).append(" (").append(index).append(")");
+      if (corrections != null) {
+        for (SpellCheckCorrection corr : corrections) {
+          sb.append("     ");
+          sb.append(corr.getOriginal()).append(">")
+              .append(corr.getCorrection()).append(" (").append(
+                  corr.getNumberOfOccurences()).append(")");
+        }
+      }
+      return sb.toString();
+    }
+  }
+  
+  private class StartOffsetComparator implements
+      Comparator<SpellCheckCorrection> {
+    @Override
+    public int compare(SpellCheckCorrection o1, SpellCheckCorrection o2) {
+      return o1.getOriginal().startOffset() - o2.getOriginal().startOffset();
+    }
+  }
+  
+  private class RankComparator implements Comparator<RankedSpellPossibility> {
+    // Rank poorer suggestions ahead of better ones for use with a PriorityQueue
+    public int compare(RankedSpellPossibility r1, RankedSpellPossibility r2) {
+      int retval = r2.rank - r1.rank;
+      if (retval == 0) {
+        retval = r2.index - r1.index;
+      }
+      return retval;
+    }
+  }
+  
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/QueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/QueryConverter.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/QueryConverter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/QueryConverter.java Mon Jun  4 17:55:25 2012
@@ -50,7 +50,32 @@ public abstract class QueryConverter imp
   private NamedList args;
 
   protected Analyzer analyzer;
-
+  
+  /**
+   * <p>This term is marked prohibited in the query with the minus sign.</p>
+   * 
+   */
+  public static final int PROHIBITED_TERM_FLAG = 16384;
+  /**
+   * <p>This term is marked required in the query with the plus sign.</p>
+   */
+  public static final int REQUIRED_TERM_FLAG = 32768;
+  /**
+   * <p>
+   * This term is directly followed by a boolean operator (AND/OR/NOT)
+   * and this operator differs from the prior boolean operator
+   * in the query (this signifies this term is likely part of a different
+   * query clause than the next term in the query)
+   * </p>
+   */
+  public static final int TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG = 65536;
+  /**
+   * <p>
+   * This term exists in a query that contains boolean operators
+   * (AND/OR/NOT)
+   * </p>
+   */
+  public static final int TERM_IN_BOOLEAN_QUERY_FLAG = 131072;
   public void init(NamedList args) {
     this.args = args;
   }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ResultEntry.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ResultEntry.java?rev=1346058&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ResultEntry.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/ResultEntry.java Mon Jun  4 17:55:25 2012
@@ -0,0 +1,57 @@
+package org.apache.solr.spelling;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.analysis.Token;
+
+public class ResultEntry {
+  public Token token;
+  public String suggestion;
+  public int freq;
+  ResultEntry(Token t, String s, int f) {
+    token = t;
+    suggestion = s;
+    freq = f;    
+  } 
+  @Override
+  public int hashCode() {  
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + freq;
+    result = prime * result
+        + ((suggestion == null) ? 0 : suggestion.hashCode());
+    result = prime * result + ((token == null) ? 0 : token.hashCode());
+    return result;
+  }
+  @Override
+  public boolean equals(Object obj) {    
+    if (this == obj) return true;
+    if (obj == null) return false;
+    if (getClass() != obj.getClass()) return false;
+    ResultEntry other = (ResultEntry) obj;
+    if (freq != other.freq) return false;
+    if (suggestion == null) {
+      if (other.suggestion != null) return false;
+    } else if (!suggestion.equals(other.suggestion)) return false;
+    if (token == null) {
+      if (other.token != null) return false;
+    } else if (!token.equals(other.token)) return false;
+    return true;
+  }
+  
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java Mon Jun  4 17:55:25 2012
@@ -92,7 +92,12 @@ public abstract class SolrSpellChecker {
       //just use .5 as a default
     }
     
-    StringDistance sd = getStringDistance() == null ? new LevensteinDistance() : getStringDistance();    
+    StringDistance sd = null;
+    try {
+      sd = getStringDistance() == null ? new LevensteinDistance() : getStringDistance();    
+    } catch(UnsupportedOperationException uoe) {
+      sd = new LevensteinDistance();
+    }
     
     SpellingResult result = new SpellingResult();
     for (Map.Entry<String, HashSet<String>> entry : mergeData.origVsSuggested.entrySet()) {
@@ -190,4 +195,8 @@ public abstract class SolrSpellChecker {
    * @throws IOException if there is an error producing suggestions
    */
   public abstract SpellingResult getSuggestions(SpellingOptions options) throws IOException;
+  
+  public boolean isSuggestionsMayOverlap() {
+    return false;
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollator.java Mon Jun  4 17:55:25 2012
@@ -37,7 +37,7 @@ public class SpellCheckCollator {
   private static final Logger LOG = LoggerFactory.getLogger(SpellCheckCollator.class);
 
   public List<SpellCheckCollation> collate(SpellingResult result, String originalQuery, ResponseBuilder ultimateResponse,
-                                           int maxCollations, int maxTries, int maxEvaluations) {
+                                           int maxCollations, int maxTries, int maxEvaluations, boolean suggestionsMayOverlap) {
     List<SpellCheckCollation> collations = new ArrayList<SpellCheckCollation>();
 
     QueryComponent queryComponent = null;
@@ -51,8 +51,10 @@ public class SpellCheckCollator {
     }
 
     boolean verifyCandidateWithQuery = true;
+    int maxNumberToIterate = maxTries;
     if (maxTries < 1) {
       maxTries = 1;
+      maxNumberToIterate = maxCollations;
       verifyCandidateWithQuery = false;
     }
     if (queryComponent == null && verifyCandidateWithQuery) {
@@ -63,11 +65,11 @@ public class SpellCheckCollator {
 
     int tryNo = 0;
     int collNo = 0;
-    PossibilityIterator possibilityIter = new PossibilityIterator(result.getSuggestions(), maxTries, maxEvaluations);
+    PossibilityIterator possibilityIter = new PossibilityIterator(result.getSuggestions(), maxNumberToIterate, maxEvaluations, suggestionsMayOverlap);
     while (tryNo < maxTries && collNo < maxCollations && possibilityIter.hasNext()) {
 
-      RankedSpellPossibility possibility = possibilityIter.next();
-      String collationQueryStr = getCollation(originalQuery, possibility.getCorrections());
+      PossibilityIterator.RankedSpellPossibility possibility = possibilityIter.next();
+      String collationQueryStr = getCollation(originalQuery, possibility.corrections);
       int hits = 0;
 
       if (verifyCandidateWithQuery) {
@@ -102,10 +104,10 @@ public class SpellCheckCollator {
         SpellCheckCollation collation = new SpellCheckCollation();
         collation.setCollationQuery(collationQueryStr);
         collation.setHits(hits);
-        collation.setInternalRank(possibility.getRank());
+        collation.setInternalRank(suggestionsMayOverlap ? ((possibility.rank * 1000) + possibility.index) : possibility.rank);
 
         NamedList<String> misspellingsAndCorrections = new NamedList<String>();
-        for (SpellCheckCorrection corr : possibility.getCorrections()) {
+        for (SpellCheckCorrection corr : possibility.corrections) {
           misspellingsAndCorrections.add(corr.getOriginal().toString(), corr.getCorrection());
         }
         collation.setMisspellingsAndCorrections(misspellingsAndCorrections);
@@ -122,16 +124,53 @@ public class SpellCheckCollator {
                               List<SpellCheckCorrection> corrections) {
     StringBuilder collation = new StringBuilder(origQuery);
     int offset = 0;
-    for (SpellCheckCorrection correction : corrections) {
+    String corr = "";
+    for(int i=0 ; i<corrections.size() ; i++) {
+      SpellCheckCorrection correction = corrections.get(i);   
       Token tok = correction.getOriginal();
       // we are replacing the query in order, but injected terms might cause
       // illegal offsets due to previous replacements.
       if (tok.getPositionIncrement() == 0)
         continue;
-      collation.replace(tok.startOffset() + offset, tok.endOffset() + offset,
-          correction.getCorrection());
-      offset += correction.getCorrection().length()
-          - (tok.endOffset() - tok.startOffset());
+      corr = correction.getCorrection();
+      boolean addParenthesis = false;
+      Character requiredOrProhibited = null;
+      int indexOfSpace = corr.indexOf(' ');
+      StringBuilder corrSb = new StringBuilder(corr);
+      int bump = 1;
+      
+      //If the correction contains whitespace (because it involved breaking a word in 2+ words),
+      //then be sure all of the new words have the same optional/required/prohibited status in the query.
+      while(indexOfSpace>-1 && indexOfSpace<corr.length()-1) {
+        addParenthesis = true;
+        char previousChar = tok.startOffset()>0 ? collation.charAt(tok.startOffset()-1) : ' ';
+        if(previousChar=='-' || previousChar=='+') {
+          corrSb.insert(indexOfSpace + bump, previousChar);
+          if(requiredOrProhibited==null) {
+            requiredOrProhibited = previousChar;
+          }
+          bump++;
+        } else if ((tok.getFlags() & QueryConverter.TERM_IN_BOOLEAN_QUERY_FLAG) == QueryConverter.TERM_IN_BOOLEAN_QUERY_FLAG) {
+          corrSb.insert(indexOfSpace + bump, "AND ");
+          bump += 4;
+        }
+        indexOfSpace = correction.getCorrection().indexOf(' ', indexOfSpace + bump);
+      }
+      
+      int oneForReqOrProhib = 0;
+      if(addParenthesis) { 
+        if(requiredOrProhibited!=null) {
+          corrSb.insert(0, requiredOrProhibited);
+          oneForReqOrProhib++;
+        }
+        corrSb.insert(0, '(');
+        corrSb.append(')');
+      }
+      corr = corrSb.toString();  
+      int startIndex = tok.startOffset() + offset - oneForReqOrProhib;
+      int endIndex = tok.endOffset() + offset;
+      collation.replace(startIndex, endIndex, corr);
+      offset += corr.length() - oneForReqOrProhib - (tok.endOffset() - tok.startOffset());      
     }
     return collation.toString();
   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SpellingQueryConverter.java Mon Jun  4 17:55:25 2012
@@ -38,8 +38,18 @@ import org.apache.lucene.analysis.tokena
 
 /**
  * Converts the query string to a Collection of Lucene tokens using a regular expression.
- * Boolean operators AND and OR are skipped.
- *
+ * Boolean operators AND, OR, NOT are skipped. 
+ * 
+ * Each term is checked to determine if it is optional, required or prohibited.  Required
+ * terms output a {@link Token} with the {@link QueryConverter#REQUIRED_TERM_FLAG} set.
+ * Prohibited terms output a {@link Token} with the {@link QueryConverter#PROHIBITED_TERM_FLAG} 
+ * set. If the query uses the plus (+) and minus (-) to denote required and prohibited, this
+ * determination will be accurate.  In the case boolean AND/OR/NOTs are used, this
+ * converter makes an uninformed guess as to whether the term would likely behave as if it
+ * is Required or Prohibited and sets the flags accordingly.  These flags are used downstream
+ * to generate collations for {@link WordBreakSolrSpellChecker}, in cases where an original 
+ * term is split up into multiple Tokens.
+ * 
  * @since solr 1.3
  **/
 public class SpellingQueryConverter extends QueryConverter  {
@@ -86,8 +96,7 @@ public class SpellingQueryConverter exte
   final static String PATTERN = "(?:(?!(" + NMTOKEN + ":|\\d+)))[\\p{L}_\\-0-9]+";
   // previous version: Pattern.compile("(?:(?!(\\w+:|\\d+)))\\w+");
   protected Pattern QUERY_REGEX = Pattern.compile(PATTERN);
-
-
+  
   /**
    * Converts the original query string to a collection of Lucene Tokens.
    * @param original the original query string
@@ -99,37 +108,87 @@ public class SpellingQueryConverter exte
       return Collections.emptyList();
     }
     Collection<Token> result = new ArrayList<Token>();
-    //TODO: Extract the words using a simple regex, but not query stuff, and then analyze them to produce the token stream
     Matcher matcher = QUERY_REGEX.matcher(original);
-    while (matcher.find()) {
-      String word = matcher.group(0);
-      if (word.equals("AND") == false && word.equals("OR") == false) {
-        try {
-          analyze(result, new StringReader(word), matcher.start());
-        } catch (IOException e) {
-          // TODO: shouldn't we log something?
-        }
+    String nextWord = null;
+    int nextStartIndex = 0;
+    String lastBooleanOp = null;
+    while (nextWord!=null || matcher.find()) {
+      String word = null;
+      int startIndex = 0;
+      if(nextWord != null) {
+        word = nextWord;
+        startIndex = nextStartIndex;
+        nextWord = null;
+      } else {
+        word = matcher.group(0);
+        startIndex = matcher.start();
+      }
+      if(matcher.find()) {
+        nextWord = matcher.group(0);
+        nextStartIndex = matcher.start();
+      }      
+      if("AND".equals(word) || "OR".equals(word) || "NOT".equals(word)) {
+        lastBooleanOp = word;        
+        continue;
+      }
+      // treat "AND NOT" as "NOT"...
+      if ("AND".equals(nextWord)
+          && original.length() > nextStartIndex + 7
+          && original.substring(nextStartIndex, nextStartIndex + 7).equals(
+              "AND NOT")) {
+        nextWord = "NOT";
+      }
+      
+      int flagValue = 0;
+      if (word.charAt(0) == '-'
+          || (startIndex > 0 && original.charAt(startIndex - 1) == '-')) {
+        flagValue = PROHIBITED_TERM_FLAG;
+      } else if (word.charAt(0) == '+'
+          || (startIndex > 0 && original.charAt(startIndex - 1) == '+')) {
+        flagValue = REQUIRED_TERM_FLAG;
+      //we don't know the default operator so just assume the first operator isn't new.
+      } else if (nextWord != null
+          && lastBooleanOp != null 
+          && !nextWord.equals(lastBooleanOp)
+          && ("AND".equals(nextWord) || "OR".equals(nextWord) || "NOT".equals(nextWord))) {
+        flagValue = TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG;
+      //...unless the 1st boolean operator is a NOT, because only AND/OR can be default.
+      } else if (nextWord != null
+          && lastBooleanOp == null
+          && !nextWord.equals(lastBooleanOp)
+          && ("NOT".equals(nextWord))) {
+        flagValue = TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG;
+      }
+      try {
+        analyze(result, new StringReader(word), startIndex, flagValue);
+      } catch (IOException e) {
+        // TODO: shouldn't we log something?
+      }   
+    }
+    if(lastBooleanOp != null) {
+      for(Token t : result) {
+        int f = t.getFlags();
+        t.setFlags(f |= QueryConverter.TERM_IN_BOOLEAN_QUERY_FLAG);
       }
     }
     return result;
   }
   
-  protected void analyze(Collection<Token> result, Reader text, int offset) throws IOException {
+  protected void analyze(Collection<Token> result, Reader text, int offset, int flagsAttValue) throws IOException {
     TokenStream stream = analyzer.tokenStream("", text);
     // TODO: support custom attributes
     CharTermAttribute termAtt = stream.addAttribute(CharTermAttribute.class);
-    FlagsAttribute flagsAtt = stream.addAttribute(FlagsAttribute.class);
     TypeAttribute typeAtt = stream.addAttribute(TypeAttribute.class);
     PayloadAttribute payloadAtt = stream.addAttribute(PayloadAttribute.class);
     PositionIncrementAttribute posIncAtt = stream.addAttribute(PositionIncrementAttribute.class);
     OffsetAttribute offsetAtt = stream.addAttribute(OffsetAttribute.class);
     stream.reset();
-    while (stream.incrementToken()) {
+    while (stream.incrementToken()) {      
       Token token = new Token();
       token.copyBuffer(termAtt.buffer(), 0, termAtt.length());
       token.setStartOffset(offset + offsetAtt.startOffset());
       token.setEndOffset(offset + offsetAtt.endOffset());
-      token.setFlags(flagsAtt.getFlags());
+      token.setFlags(flagsAttValue); //overwriting any flags already set...
       token.setType(typeAtt.type());
       token.setPayload(payloadAtt.getPayload());
       token.setPositionIncrement(posIncAtt.getPositionIncrement());

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/SuggestQueryConverter.java Mon Jun  4 17:55:25 2012
@@ -38,7 +38,7 @@ public class SuggestQueryConverter exten
 
     Collection<Token> result = new ArrayList<Token>();
     try {
-      analyze(result, new StringReader(original), 0);
+      analyze(result, new StringReader(original), 0, 0);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/WordBreakSolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/WordBreakSolrSpellChecker.java?rev=1346058&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/WordBreakSolrSpellChecker.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/spelling/WordBreakSolrSpellChecker.java Mon Jun  4 17:55:25 2012
@@ -0,0 +1,322 @@
+package org.apache.solr.spelling;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.spell.CombineSuggestion;
+import org.apache.lucene.search.spell.SuggestWord;
+import org.apache.lucene.search.spell.WordBreakSpellChecker;
+import org.apache.lucene.search.spell.WordBreakSpellChecker.BreakSuggestionSortMethod;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.search.SolrIndexSearcher;
+
+/**
+ * <p>
+ * A spellchecker that breaks and combines words.  
+ * </p>
+ * <p>
+ * This will not combine adjacent tokens that do not have 
+ * the same required status (prohibited, required, optional).  
+ * However, this feature depends on incoming term flags 
+ * being properly set. ({@link QueryConverter#PROHIBITED_TERM_FLAG},
+ * {@link QueryConverter#REQUIRED_TERM_FLAG}, 
+ * {@link QueryConverter#TERM_IN_BOOLEAN_QUERY_FLAG}, and
+ * {@link QueryConverter#TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG} )
+ * This feature breaks completely if the upstream analyzer or query
+ * converter sets flags with the same values but different meanings.
+ * The default query converter (if not using "spellcheck.q") 
+ * is {@link SpellingQueryConverter}, which properly sets these flags.
+ * </p>
+ */
+public class WordBreakSolrSpellChecker extends SolrSpellChecker {
+  /**
+   * <p>
+   * Try to combine multiple words into one? [true|false]
+   * </p>
+   */
+  public static final String PARAM_COMBINE_WORDS = "combineWords";
+  /**
+   * <p>
+   * Try to break words into multiples? [true|false]
+   * </p>
+   */
+  public static final String PARAM_BREAK_WORDS = "breakWords";
+  /**
+   * See {@link WordBreakSpellChecker#setMaxChanges}
+   */
+  public static final String PARAM_MAX_CHANGES = "maxChanges";
+  /**
+   * See {@link WordBreakSpellChecker#setMaxCombineWordLength}
+   */
+  public static final String PARAM_MAX_COMBINE_WORD_LENGTH = "maxCombinedLength";
+  /**
+   * See {@link WordBreakSpellChecker#setMinBreakWordLength}
+   */
+  public static final String PARAM_MIN_BREAK_WORD_LENGTH = "minBreakLength";
+  /**
+   * See {@link BreakSuggestionTieBreaker} for options.
+   */
+  public static final String PARAM_BREAK_SUGGESTION_TIE_BREAKER = "breakSugestionTieBreaker";
+  /**
+   * See {@link WordBreakSpellChecker#setMaxEvaluations}
+   */
+  public static final String PARAM_MAX_EVALUATIONS = "maxEvaluations";
+  /**
+   * See {@link WordBreakSpellChecker#setMinSuggestionFrequency}
+   */
+  public static final String PARAM_MIN_SUGGESTION_FREQUENCY = "minSuggestionFreq";
+  
+  public enum BreakSuggestionTieBreaker {
+    /**
+     * See
+     * {@link WordBreakSpellChecker.BreakSuggestionSortMethod#NUM_CHANGES_THEN_MAX_FREQUENCY}
+     * #
+     */
+    MAX_FREQ,
+    /**
+     * See
+     * {@link WordBreakSpellChecker.BreakSuggestionSortMethod#NUM_CHANGES_THEN_SUMMED_FREQUENCY}
+     */
+    SUM_FREQ
+  };
+  
+  private WordBreakSpellChecker wbsp = null;
+  private boolean combineWords = false;
+  private boolean breakWords = false;
+  private BreakSuggestionSortMethod sortMethod = BreakSuggestionSortMethod.NUM_CHANGES_THEN_MAX_FREQUENCY;
+
+  @Override
+  public String init(@SuppressWarnings("unchecked") NamedList config,
+      SolrCore core) {
+    String name = super.init(config, core);
+    combineWords = boolParam(config, PARAM_COMBINE_WORDS);
+    breakWords = boolParam(config, PARAM_BREAK_WORDS);
+    wbsp = new WordBreakSpellChecker();
+    String bstb = strParam(config, PARAM_BREAK_SUGGESTION_TIE_BREAKER);
+    if (bstb != null) {
+      bstb = bstb.toUpperCase();
+      if (bstb.equals(BreakSuggestionTieBreaker.SUM_FREQ.name())) {
+        sortMethod = BreakSuggestionSortMethod.NUM_CHANGES_THEN_SUMMED_FREQUENCY;
+      } else if (bstb.equals(BreakSuggestionTieBreaker.MAX_FREQ.name())) {
+        sortMethod = BreakSuggestionSortMethod.NUM_CHANGES_THEN_MAX_FREQUENCY;
+      } else {
+        throw new IllegalArgumentException("Invalid value for parameter "
+            + PARAM_BREAK_SUGGESTION_TIE_BREAKER + " : " + bstb);
+      }
+    }
+    int mc = intParam(config, PARAM_MAX_CHANGES);
+    if (mc > 0) {
+      wbsp.setMaxChanges(mc);
+    }
+    int mcl = intParam(config, PARAM_MAX_COMBINE_WORD_LENGTH);
+    if (mcl > 0) {
+      wbsp.setMaxCombineWordLength(mcl);
+    }
+    int mbwl = intParam(config, PARAM_MIN_BREAK_WORD_LENGTH);
+    if (mbwl > 0) {
+      wbsp.setMinBreakWordLength(mbwl);
+    }
+    int me = intParam(config, PARAM_MAX_EVALUATIONS);
+    if (me > 0) {
+      wbsp.setMaxEvaluations(me);
+    }
+    int msf = intParam(config, PARAM_MIN_SUGGESTION_FREQUENCY);
+    if (msf > 0) {
+      wbsp.setMinSuggestionFrequency(msf);
+    }
+    return name;
+  }
+  
+  private String strParam(@SuppressWarnings("unchecked") NamedList config,
+      String paramName) {
+    Object o = config.get(paramName);
+    return o == null ? null : o.toString();
+  }
+  
+  private boolean boolParam(@SuppressWarnings("unchecked") NamedList config,
+      String paramName) {
+    String s = strParam(config, paramName);
+    if ("true".equalsIgnoreCase(s) || "on".equalsIgnoreCase(s)) {
+      return true;
+    }
+    return false;
+  }
+  
+  private int intParam(@SuppressWarnings("unchecked") NamedList config,
+      String paramName) {
+    Object o = config.get(paramName);
+    if (o == null) {
+      return 0;
+    }
+    try {
+      return Integer.parseInt(o.toString());
+    } catch (NumberFormatException nfe) {
+      throw new IllegalArgumentException("Invalid integer for parameter "
+          + paramName + " : " + o);
+    }
+  }
+  
+  @Override
+  public SpellingResult getSuggestions(SpellingOptions options)
+      throws IOException {
+    IndexReader ir = options.reader;
+    int numSuggestions = options.count;
+    
+    StringBuilder sb = new StringBuilder();
+    Token[] tokenArr = options.tokens.toArray(new Token[options.tokens.size()]);
+    List<Term> termArr = new ArrayList<Term>(options.tokens.size() + 2);
+    
+    List<ResultEntry> breakSuggestionList = new ArrayList<ResultEntry>();
+    boolean lastOneProhibited = false;
+    boolean lastOneRequired = false;
+    boolean lastOneprocedesNewBooleanOp = false;
+    for (int i = 0; i < tokenArr.length; i++) {      
+      boolean prohibited = 
+        (tokenArr[i].getFlags() & QueryConverter.PROHIBITED_TERM_FLAG) == 
+          QueryConverter.PROHIBITED_TERM_FLAG;
+      boolean required = 
+        (tokenArr[i].getFlags() & QueryConverter.REQUIRED_TERM_FLAG) == 
+          QueryConverter.REQUIRED_TERM_FLAG;
+      boolean procedesNewBooleanOp = 
+        (tokenArr[i].getFlags() & QueryConverter.TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG) == 
+          QueryConverter.TERM_PRECEDES_NEW_BOOLEAN_OPERATOR_FLAG;
+      if (i > 0
+          && (prohibited != lastOneProhibited || required != lastOneRequired || lastOneprocedesNewBooleanOp)) {
+        termArr.add(WordBreakSpellChecker.SEPARATOR_TERM);
+      }
+      lastOneProhibited = prohibited;
+      lastOneRequired = required;
+      lastOneprocedesNewBooleanOp = procedesNewBooleanOp;
+      
+      Term thisTerm = new Term(field, tokenArr[i].toString());
+      termArr.add(thisTerm);
+      if (breakWords) {
+        SuggestWord[][] breakSuggestions = wbsp.suggestWordBreaks(thisTerm,
+            numSuggestions, ir, options.suggestMode, sortMethod);
+        for (SuggestWord[] breakSuggestion : breakSuggestions) {
+          sb.delete(0, sb.length());
+          boolean firstOne = true;
+          int freq = 0;
+          for (SuggestWord word : breakSuggestion) {
+            if (!firstOne) {
+              sb.append(" ");
+            }
+            firstOne = false;
+            sb.append(word.string);
+            if (sortMethod == BreakSuggestionSortMethod.NUM_CHANGES_THEN_MAX_FREQUENCY) {
+              freq = Math.max(freq, word.freq);
+            } else {
+              freq += word.freq;
+            }
+          }
+          breakSuggestionList.add(new ResultEntry(tokenArr[i], sb.toString(),
+              freq));
+        }
+      }
+    }    
+    List<ResultEntry> combineSuggestionList = Collections.emptyList();
+    CombineSuggestion[] combineSuggestions = wbsp.suggestWordCombinations(
+        termArr.toArray(new Term[termArr.size()]), numSuggestions, ir, options.suggestMode);
+    if (combineWords) {
+      combineSuggestionList = new ArrayList<ResultEntry>(
+          combineSuggestions.length);
+      for (CombineSuggestion cs : combineSuggestions) {
+        int firstTermIndex = cs.originalTermIndexes[0];
+        int lastTermIndex = cs.originalTermIndexes[cs.originalTermIndexes.length - 1];
+        sb.delete(0, sb.length());
+        for (int i = firstTermIndex; i <= lastTermIndex; i++) {
+          if (i > firstTermIndex) {
+            sb.append(" ");
+          }
+          sb.append(tokenArr[i].toString());
+        }
+        Token token = new Token(sb.toString(), tokenArr[firstTermIndex]
+            .startOffset(), tokenArr[lastTermIndex].endOffset());
+        combineSuggestionList.add(new ResultEntry(token, cs.suggestion.string,
+            cs.suggestion.freq));
+      }
+    }
+    
+    // Interleave the two lists of suggestions into one SpellingResult
+    SpellingResult result = new SpellingResult();
+    Iterator<ResultEntry> breakIter = breakSuggestionList.iterator();
+    Iterator<ResultEntry> combineIter = combineSuggestionList.iterator();
+    ResultEntry lastBreak = breakIter.hasNext() ? breakIter.next() : null;
+    ResultEntry lastCombine = combineIter.hasNext() ? combineIter.next() : null;
+    int breakCount = 0;
+    int combineCount = 0;
+    while (lastBreak != null || lastCombine != null) {
+      if (lastBreak == null) {
+        result.add(lastCombine.token, lastCombine.suggestion, lastCombine.freq);
+        lastCombine = null;
+      } else if (lastCombine == null) {
+        result.add(lastBreak.token, lastBreak.suggestion, lastBreak.freq);
+        lastBreak = null;
+      } else if (lastBreak.freq < lastCombine.freq) {
+        result.add(lastCombine.token, lastCombine.suggestion, lastCombine.freq);
+        lastCombine = null;
+      } else if (lastCombine.freq < lastBreak.freq) {
+        result.add(lastBreak.token, lastBreak.suggestion, lastBreak.freq);
+        lastBreak = null;
+      } else if (breakCount >= combineCount) {
+        result.add(lastCombine.token, lastCombine.suggestion, lastCombine.freq);
+        lastCombine = null;
+      } else {
+        result.add(lastBreak.token, lastBreak.suggestion, lastBreak.freq);
+        lastBreak = null;
+      }
+      if (result.getSuggestions().size() > numSuggestions) {
+        break;
+      }
+      if (lastBreak == null && breakIter.hasNext()) {
+        lastBreak = breakIter.next();
+        breakCount++;
+      }
+      if (lastCombine == null && combineIter.hasNext()) {
+        lastCombine = combineIter.next();
+        combineCount++;
+      }
+    }
+    return result;
+  }
+  
+  @Override
+  public void build(SolrCore core, SolrIndexSearcher searcher) {
+  /* no-op */
+  }
+  
+  @Override
+  public void reload(SolrCore core, SolrIndexSearcher searcher)
+      throws IOException {
+  /* no-op */
+  }
+  
+  @Override
+  public boolean isSuggestionsMayOverlap() {
+    return true;
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-spellcheckcomponent.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-spellcheckcomponent.xml?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-spellcheckcomponent.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-spellcheckcomponent.xml Mon Jun  4 17:55:25 2012
@@ -70,6 +70,14 @@ Config for testing spellcheck component
       <str name="field">lowerfilt</str>
     </lst>
     <lst name="spellchecker">
+      <str name="name">wordbreak</str>
+      <str name="classname">solr.WordBreakSolrSpellChecker</str>      
+      <str name="field">lowerfilt</str>
+      <str name="combineWords">true</str>
+      <str name="breakWords">true</str>
+      <int name="maxChanges">10</int>
+    </lst>
+    <lst name="spellchecker">
       <str name="name">threshold</str>
       <str name="field">lowerfilt</str>
       <str name="spellcheckIndexDir">spellcheckerThreshold</str>
@@ -161,5 +169,15 @@ Config for testing spellcheck component
 				<str>spellcheck</str>
 			</arr>
  </requestHandler>
+ <requestHandler name="spellCheckWithWordbreak" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">default</str>
+      <str name="spellcheck.dictionary">wordbreak</str>
+      <str name="spellcheck.count">20</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
 
 </config>

Modified: lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig.xml?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig.xml (original)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig.xml Mon Jun  4 17:55:25 2012
@@ -280,6 +280,14 @@
       <int name="minQueryLength">3</int>
     </lst>
     <lst name="spellchecker">
+      <str name="name">wordbreak</str>
+      <str name="classname">solr.WordBreakSolrSpellChecker</str>      
+      <str name="field">lowerfilt</str>
+      <str name="combineWords">true</str>
+      <str name="breakWords">true</str>
+      <int name="maxChanges">10</int>
+    </lst>
+    <lst name="spellchecker">
 			<str name="name">multipleFields</str>
 			<str name="field">lowerfilt1and2</str>
 			<str name="spellcheckIndexDir">spellcheckerMultipleFields</str>
@@ -365,6 +373,26 @@
       <str>spellcheck</str>
     </arr>
   </requestHandler>
+  <requestHandler name="spellCheckWithWordbreak" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">default</str>
+      <str name="spellcheck.dictionary">wordbreak</str>
+      <str name="spellcheck.count">20</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
+  <requestHandler name="spellCheckWithWordbreak_Direct" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">direct</str>
+      <str name="spellcheck.dictionary">wordbreak</str>
+      <str name="spellcheck.count">20</str>
+    </lst>
+    <arr name="last-components">
+      <str>spellcheck</str>
+    </arr>
+  </requestHandler>
   <requestHandler name="spellCheckCompRH1" class="org.apache.solr.handler.component.SearchHandler">
 			<lst name="defaults">
 				<str name="defType">dismax</str>

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java Mon Jun  4 17:55:25 2012
@@ -37,6 +37,7 @@ import org.apache.solr.common.util.Named
 public class DistributedSpellCheckComponentTest extends BaseDistributedSearchTestCase {
   
   private String requestHandlerName;
+  private String reqHandlerWithWordbreak;
   
 	public DistributedSpellCheckComponentTest()
 	{
@@ -52,7 +53,13 @@ public class DistributedSpellCheckCompon
     // this test requires FSDir
     saveProp = System.getProperty("solr.directoryFactory");
     System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");    
-    requestHandlerName = random().nextBoolean() ? "spellCheckCompRH" : "spellCheckCompRH_Direct"; 
+    if(random().nextBoolean()) {
+      requestHandlerName = "spellCheckCompRH";
+      reqHandlerWithWordbreak = "spellCheckWithWordbreak";      
+    } else {
+      requestHandlerName = "spellCheckCompRH_Direct";
+      reqHandlerWithWordbreak = "spellCheckWithWordbreak_Direct";
+    }  
     super.setUp();
   }
   
@@ -141,5 +148,7 @@ public class DistributedSpellCheckCompon
   
     query("q", "lowerfilt:(\"quote red fox\")", "fl", "id,lowerfilt", "spellcheck", "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH", SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "10", SpellCheckComponent.SPELLCHECK_COLLATE, "true", SpellCheckComponent.SPELLCHECK_MAX_COLLATION_TRIES, "10", SpellCheckComponent.SPELLCHECK_MAX_COLLATIONS, "1", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_ALTERNATIVE_TERM_COUNT, "5", SpellCheckComponent.SPELLCHECK_MAX_RESULTS_FOR_SUGGEST, "10");
     query("q", "lowerfilt:(\"rod fix\")", "fl", "id,lowerfilt", "spellcheck", "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH", SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "10", SpellCheckComponent.SPELLCHECK_COLLATE, "true", SpellCheckComponent.SPELLCHECK_MAX_COLLATION_TRIES, "10", SpellCheckComponent.SPELLCHECK_MAX_COLLATIONS, "1", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_ALTERNATIVE_TERM_COUNT, "5", SpellCheckComponent.SPELLCHECK_MAX_RESULTS_FOR_SUGGEST, "10");
+  
+    query("q", "lowerfilt:(+quock +redfox +jum +ped)", "fl", "id,lowerfilt", "spellcheck", "true", "qt", reqHandlerWithWordbreak, "shards.qt", reqHandlerWithWordbreak, SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "10", SpellCheckComponent.SPELLCHECK_COLLATE, "true", SpellCheckComponent.SPELLCHECK_MAX_COLLATION_TRIES, "0", SpellCheckComponent.SPELLCHECK_MAX_COLLATIONS, "1", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true");
   }
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellPossibilityIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellPossibilityIteratorTest.java?rev=1346058&r1=1346057&r2=1346058&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellPossibilityIteratorTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/spelling/SpellPossibilityIteratorTest.java Mon Jun  4 17:55:25 2012
@@ -16,8 +16,10 @@ package org.apache.solr.spelling;
  * limitations under the License.
  */
 
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.lucene.analysis.Token;
 import org.apache.solr.SolrTestCaseJ4;
@@ -26,17 +28,22 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class SpellPossibilityIteratorTest extends SolrTestCaseJ4 {
-
-	private static Map<Token, LinkedHashMap<String, Integer>> suggestions = new LinkedHashMap<Token, LinkedHashMap<String, Integer>>();
-	private static Map<Token, LinkedHashMap<String, Integer>> lotsaSuggestions = new LinkedHashMap<Token, LinkedHashMap<String, Integer>>();
-
+	private static final Token TOKEN_AYE = new Token("AYE", 0, 3);
+	private static final Token TOKEN_BEE = new Token("BEE", 4, 7);
+	private static final Token TOKEN_AYE_BEE = new Token("AYE BEE", 0, 7);
+	private static final Token TOKEN_CEE = new Token("CEE", 8, 11);
+	
+	private LinkedHashMap<String, Integer> AYE;
+	private LinkedHashMap<String, Integer> BEE;
+	private LinkedHashMap<String, Integer> AYE_BEE;
+	private LinkedHashMap<String, Integer> CEE;
+	
 	@Override
   @Before
 	public void setUp() throws Exception {
 	  super.setUp();
-		suggestions.clear();
 
-		LinkedHashMap<String, Integer> AYE = new LinkedHashMap<String, Integer>();
+		AYE = new LinkedHashMap<String, Integer>();
 		AYE.put("I", 0);
 		AYE.put("II", 0);
 		AYE.put("III", 0);
@@ -46,7 +53,7 @@ public class SpellPossibilityIteratorTes
 		AYE.put("VII", 0);
 		AYE.put("VIII", 0);
 		
-		LinkedHashMap<String, Integer> BEE = new LinkedHashMap<String, Integer>();
+		BEE = new LinkedHashMap<String, Integer>();
 		BEE.put("alpha", 0);
 		BEE.put("beta", 0);
 		BEE.put("gamma", 0);
@@ -57,8 +64,19 @@ public class SpellPossibilityIteratorTes
 		BEE.put("theta", 0);
 		BEE.put("iota", 0);
 		
+		AYE_BEE = new LinkedHashMap<String, Integer>();
+		AYE_BEE.put("one-alpha", 0);
+		AYE_BEE.put("two-beta", 0);
+		AYE_BEE.put("three-gamma", 0);
+		AYE_BEE.put("four-delta", 0);
+		AYE_BEE.put("five-epsilon", 0);
+		AYE_BEE.put("six-zeta", 0);
+		AYE_BEE.put("seven-eta", 0);
+		AYE_BEE.put("eight-theta", 0);
+		AYE_BEE.put("nine-iota", 0);
+		
 
-		LinkedHashMap<String, Integer> CEE = new LinkedHashMap<String, Integer>();
+		CEE = new LinkedHashMap<String, Integer>();
 		CEE.put("one", 0);
 		CEE.put("two", 0);
 		CEE.put("three", 0);
@@ -69,61 +87,75 @@ public class SpellPossibilityIteratorTes
 		CEE.put("eight", 0);
 		CEE.put("nine", 0);
 		CEE.put("ten", 0);
-
-		suggestions.put(new Token("AYE", 0, 2), AYE);
-		suggestions.put(new Token("BEE", 0, 2), BEE);
-		suggestions.put(new Token("CEE", 0, 2), CEE);
-		
-		lotsaSuggestions.put(new Token("AYE", 0, 2), AYE);
-		lotsaSuggestions.put(new Token("BEE", 0, 2), BEE);
-		lotsaSuggestions.put(new Token("CEE", 0, 2), CEE);
-		
-		lotsaSuggestions.put(new Token("AYE1", 0, 3), AYE);
-		lotsaSuggestions.put(new Token("BEE1", 0, 3), BEE);
-		lotsaSuggestions.put(new Token("CEE1", 0, 3), CEE);
-		
-		lotsaSuggestions.put(new Token("AYE2", 0, 3), AYE);
-		lotsaSuggestions.put(new Token("BEE2", 0, 3), BEE);
-		lotsaSuggestions.put(new Token("CEE2", 0, 3), CEE);
-		
-		lotsaSuggestions.put(new Token("AYE3", 0, 3), AYE);
-		lotsaSuggestions.put(new Token("BEE3", 0, 3), BEE);
-		lotsaSuggestions.put(new Token("CEE3", 0, 3), CEE);
-		
-		lotsaSuggestions.put(new Token("AYE4", 0, 3), AYE);
-		lotsaSuggestions.put(new Token("BEE4", 0, 3), BEE);
-		lotsaSuggestions.put(new Token("CEE4", 0, 3), CEE);
 	}
 	
 	@Test
 	public void testScalability() throws Exception {
-		PossibilityIterator iter = new PossibilityIterator(lotsaSuggestions, 1000, 10000);
+	  Map<Token, LinkedHashMap<String, Integer>> lotsaSuggestions = new LinkedHashMap<Token, LinkedHashMap<String, Integer>>();
+	  lotsaSuggestions.put(TOKEN_AYE , AYE);
+    lotsaSuggestions.put(TOKEN_BEE , BEE);
+    lotsaSuggestions.put(TOKEN_CEE , CEE);
+    
+    lotsaSuggestions.put(new Token("AYE1", 0, 3),  AYE);
+    lotsaSuggestions.put(new Token("BEE1", 4, 7),  BEE);
+    lotsaSuggestions.put(new Token("CEE1", 8, 11), CEE);
+    
+    lotsaSuggestions.put(new Token("AYE2", 0, 3),  AYE);
+    lotsaSuggestions.put(new Token("BEE2", 4, 7),  BEE);
+    lotsaSuggestions.put(new Token("CEE2", 8, 11), CEE);
+    
+    lotsaSuggestions.put(new Token("AYE3", 0, 3),  AYE);
+    lotsaSuggestions.put(new Token("BEE3", 4, 7),  BEE);
+    lotsaSuggestions.put(new Token("CEE3", 8, 11), CEE);
+    
+    lotsaSuggestions.put(new Token("AYE4", 0, 3),  AYE);
+    lotsaSuggestions.put(new Token("BEE4", 4, 7),  BEE);
+    lotsaSuggestions.put(new Token("CEE4", 8, 11), CEE);
+    
+		PossibilityIterator iter = new PossibilityIterator(lotsaSuggestions, 1000, 10000, false);
 		int count = 0;
 		while (iter.hasNext()) {			
-			RankedSpellPossibility rsp = iter.next();
+			PossibilityIterator.RankedSpellPossibility rsp = iter.next();
 			count++;
 		}
 		assertTrue(count==1000);
+		
+		lotsaSuggestions.put(new Token("AYE_BEE1", 0, 7), AYE_BEE);
+    lotsaSuggestions.put(new Token("AYE_BEE2", 0, 7), AYE_BEE);
+    lotsaSuggestions.put(new Token("AYE_BEE3", 0, 7), AYE_BEE);
+    lotsaSuggestions.put(new Token("AYE_BEE4", 0, 7), AYE_BEE);
+    iter = new PossibilityIterator(lotsaSuggestions, 1000, 10000, true);
+    count = 0;
+    while (iter.hasNext()) {      
+      PossibilityIterator.RankedSpellPossibility rsp = iter.next();
+      count++;
+    }
+    assertTrue(count<100);
 	}
 	
 	@Test
 	public void testSpellPossibilityIterator() throws Exception {
-		PossibilityIterator iter = new PossibilityIterator(suggestions, 1000, 10000);
+	  Map<Token, LinkedHashMap<String, Integer>> suggestions = new LinkedHashMap<Token, LinkedHashMap<String, Integer>>();
+	  suggestions.put(TOKEN_AYE , AYE);
+    suggestions.put(TOKEN_BEE , BEE);
+    suggestions.put(TOKEN_CEE , CEE);
+    
+		PossibilityIterator iter = new PossibilityIterator(suggestions, 1000, 10000, false);
 		int count = 0;
 		while (iter.hasNext()) {
 			
-			RankedSpellPossibility rsp = iter.next();
+		  PossibilityIterator.RankedSpellPossibility rsp = iter.next();
 			if(count==0) {
-				assertTrue("I".equals(rsp.getCorrections().get(0).getCorrection()));
-				assertTrue("alpha".equals(rsp.getCorrections().get(1).getCorrection()));
-				assertTrue("one".equals(rsp.getCorrections().get(2).getCorrection()));
+				assertTrue("I".equals(rsp.corrections.get(0).getCorrection()));
+				assertTrue("alpha".equals(rsp.corrections.get(1).getCorrection()));
+				assertTrue("one".equals(rsp.corrections.get(2).getCorrection()));
 			}
 			count++;
 		}
 		assertTrue(("Three maps (8*9*10) should return 720 iterations but instead returned " + count), count == 720);
 
-		suggestions.remove(new Token("CEE", 0, 2));
-		iter = new PossibilityIterator(suggestions, 100, 10000);
+		suggestions.remove(TOKEN_CEE);
+		iter = new PossibilityIterator(suggestions, 100, 10000, false);
 		count = 0;
 		while (iter.hasNext()) {
 			iter.next();
@@ -131,8 +163,8 @@ public class SpellPossibilityIteratorTes
 		}
 		assertTrue(("Two maps (8*9) should return 72 iterations but instead returned " + count), count == 72);
 
-		suggestions.remove(new Token("BEE", 0, 2));
-		iter = new PossibilityIterator(suggestions, 5, 10000);
+		suggestions.remove(TOKEN_BEE);
+		iter = new PossibilityIterator(suggestions, 5, 10000, false);
 		count = 0;
 		while (iter.hasNext()) {
 			iter.next();
@@ -140,8 +172,8 @@ public class SpellPossibilityIteratorTes
 		}
 		assertTrue(("We requested 5 suggestions but got " + count), count == 5);
 
-		suggestions.remove(new Token("AYE", 0, 2));
-		iter = new PossibilityIterator(suggestions, Integer.MAX_VALUE, 10000);
+		suggestions.remove(TOKEN_AYE);
+		iter = new PossibilityIterator(suggestions, Integer.MAX_VALUE, 10000, false);
 		count = 0;
 		while (iter.hasNext()) {
 			iter.next();
@@ -150,4 +182,47 @@ public class SpellPossibilityIteratorTes
 		assertTrue(("No maps should return 0 iterations but instead returned " + count), count == 0);
 
 	}
+	
+	@Test
+  public void testOverlappingTokens() throws Exception {
+	  Map<Token, LinkedHashMap<String, Integer>> overlappingSuggestions = new LinkedHashMap<Token, LinkedHashMap<String, Integer>>();
+	  overlappingSuggestions.put(TOKEN_AYE, AYE);
+    overlappingSuggestions.put(TOKEN_BEE, BEE);
+    overlappingSuggestions.put(TOKEN_AYE_BEE, AYE_BEE);
+    overlappingSuggestions.put(TOKEN_CEE, CEE);
+    
+    PossibilityIterator iter = new PossibilityIterator(overlappingSuggestions, Integer.MAX_VALUE, Integer.MAX_VALUE, true);
+    int aCount = 0;
+    int abCount = 0;
+    Set<PossibilityIterator.RankedSpellPossibility> dupChecker = new HashSet<PossibilityIterator.RankedSpellPossibility>();
+    while (iter.hasNext()) {
+      PossibilityIterator.RankedSpellPossibility rsp = iter.next();
+      Token a = null;
+      Token b = null;
+      Token ab = null;
+      Token c = null;
+      for(SpellCheckCorrection scc : rsp.corrections) {
+        if(scc.getOriginal().equals(TOKEN_AYE)) {
+          a = scc.getOriginal();
+        } else if(scc.getOriginal().equals(TOKEN_BEE)) {
+          b = scc.getOriginal();
+        } else if(scc.getOriginal().equals(TOKEN_AYE_BEE)) {
+          ab = scc.getOriginal();
+        } else if(scc.getOriginal().equals(TOKEN_CEE)) {
+          c = scc.getOriginal();
+        }       
+        if(ab!=null) {
+          abCount++;
+        } else {
+          aCount++;
+        }       
+      }
+      assertTrue(c != null);
+      assertTrue(ab != null || (a!=null && b!=null));
+      assertTrue(ab == null || (a==null && b==null));
+      assertTrue(dupChecker.add(rsp));
+    }
+    assertTrue(aCount==2160);
+    assertTrue(abCount==180);
+  }
 }