You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/11/11 14:08:42 UTC

svn commit: r1200865 [4/4] - in /lucene/dev/branches/lucene2621: ./ dev-tools/idea/lucene/contrib/ lucene/ lucene/contrib/ lucene/contrib/misc/src/java/org/apache/lucene/index/ lucene/contrib/misc/src/java/org/apache/lucene/search/ lucene/contrib/misc/...

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Fri Nov 11 13:08:38 2011
@@ -22,6 +22,8 @@ import java.io.StringReader;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.lucene.search.spell.DirectSpellChecker;
+import org.apache.lucene.search.spell.JaroWinklerDistance;
 import org.apache.lucene.search.spell.LevensteinDistance;
 import org.apache.lucene.search.spell.StringDistance;
 import org.apache.lucene.search.spell.SuggestWord;
@@ -147,7 +149,7 @@ public class SpellCheckComponent extends
         IndexReader reader = rb.req.getSearcher().getIndexReader();
         boolean collate = params.getBool(SPELLCHECK_COLLATE, false);
         float accuracy = params.getFloat(SPELLCHECK_ACCURACY, Float.MIN_VALUE);
-        SolrParams customParams = getCustomParams(getDictionaryName(params), params, shardRequest);
+        SolrParams customParams = getCustomParams(getDictionaryName(params), params);
         SpellingOptions options = new SpellingOptions(tokens, reader, count, onlyMorePopular, extendedResults,
                 accuracy, customParams);                       
         SpellingResult spellingResult = spellChecker.getSuggestions(options);
@@ -210,7 +212,7 @@ public class SpellCheckComponent extends
    * @param params The original SolrParams
    * @return The new Params
    */
-  protected SolrParams getCustomParams(String dictionary, SolrParams params, boolean shardRequest) {
+  protected SolrParams getCustomParams(String dictionary, SolrParams params) {
     ModifiableSolrParams result = new ModifiableSolrParams();
     Iterator<String> iter = params.getParameterNamesIterator();
     String prefix = SpellingParams.SPELLCHECK_PREFIX + "." + dictionary + ".";
@@ -220,10 +222,6 @@ public class SpellCheckComponent extends
         result.add(nxt.substring(prefix.length()), params.getParams(nxt));
       }
     }
-    if(shardRequest)
-    {
-    	result.add(ShardParams.IS_SHARD, "true");
-    }
     return result;
   }
 
@@ -256,6 +254,8 @@ public class SpellCheckComponent extends
     boolean collationExtendedResults = params.getBool(SPELLCHECK_COLLATE_EXTENDED_RESULTS, false);
     int maxCollationTries = params.getInt(SPELLCHECK_MAX_COLLATION_TRIES, 0);
     int maxCollations = params.getInt(SPELLCHECK_MAX_COLLATIONS, 1);
+    int count = rb.req.getParams().getInt(SPELLCHECK_COUNT, 1);
+    int numSug = Math.max(count, AbstractLuceneSpellChecker.DEFAULT_SUGGESTION_COUNT);
 
     String origQuery = params.get(SPELLCHECK_Q);
     if (origQuery == null) {
@@ -263,192 +263,30 @@ public class SpellCheckComponent extends
       if (origQuery == null) {
         origQuery = params.get(CommonParams.Q);
       }
-    }
-
-    int count = rb.req.getParams().getInt(SPELLCHECK_COUNT, 1);
-    float min = 0.5f;
-    StringDistance sd = null;
-    int numSug = Math.max(count, AbstractLuceneSpellChecker.DEFAULT_SUGGESTION_COUNT);
-    SolrSpellChecker checker = getSpellChecker(rb.req.getParams());
-    if (checker instanceof AbstractLuceneSpellChecker) {
-      AbstractLuceneSpellChecker spellChecker = (AbstractLuceneSpellChecker) checker;
-      min = spellChecker.getAccuracy();
-      sd = spellChecker.getStringDistance();
-    }
-    if (sd == null)
-      sd = new LevensteinDistance();
-
-    Collection<Token> tokens = null;
-    try {
-      tokens = getTokens(origQuery, checker.getQueryAnalyzer());
-    } catch (IOException e) {
-      LOG.error("Could not get tokens (this should never happen)", e);
-    }
-
-    // original token -> corresponding Suggestion object (keep track of start,end)
-    Map<String, SpellCheckResponse.Suggestion> origVsSuggestion = new HashMap<String, SpellCheckResponse.Suggestion>();
-    // original token string -> summed up frequency
-    Map<String, Integer> origVsFreq = new HashMap<String, Integer>();
-    // original token string -> # of shards reporting it as misspelled
-    Map<String, Integer> origVsShards = new HashMap<String, Integer>();
-    // original token string -> set of alternatives
-    // must preserve order because collation algorithm can only work in-order
-    Map<String, HashSet<String>> origVsSuggested = new LinkedHashMap<String, HashSet<String>>();
-    // alternative string -> corresponding SuggestWord object
-    Map<String, SuggestWord> suggestedVsWord = new HashMap<String, SuggestWord>();
-    Map<String, SpellCheckCollation> collations = new HashMap<String, SpellCheckCollation>();
+    }    
     
-    int totalNumberShardResponses = 0;
+    SpellCheckMergeData mergeData = new SpellCheckMergeData();     
     for (ShardRequest sreq : rb.finished) {
       for (ShardResponse srsp : sreq.responses) {
         NamedList nl = (NamedList) srsp.getSolrResponse().getResponse().get("spellcheck");
         LOG.info(srsp.getShard() + " " + nl);
         if (nl != null) {
-        	totalNumberShardResponses++;
-          SpellCheckResponse spellCheckResp = new SpellCheckResponse(nl);
-          for (SpellCheckResponse.Suggestion suggestion : spellCheckResp.getSuggestions()) {
-            origVsSuggestion.put(suggestion.getToken(), suggestion);
-            HashSet<String> suggested = origVsSuggested.get(suggestion.getToken());
-            if (suggested == null) {
-              suggested = new HashSet<String>();
-              origVsSuggested.put(suggestion.getToken(), suggested);
-            }
-
-            // sum up original frequency          
-            int origFreq = 0;
-            Integer o = origVsFreq.get(suggestion.getToken());
-            if (o != null)  origFreq += o;
-            origFreq += suggestion.getOriginalFrequency();
-            origVsFreq.put(suggestion.getToken(), origFreq);
-            
-            //# shards reporting
-            Integer origShards = origVsShards.get(suggestion.getToken());
-            if(origShards==null) {
-            	origVsShards.put(suggestion.getToken(), 1);
-            } else {
-            	origVsShards.put(suggestion.getToken(), ++origShards);
-            }            
-
-            // find best suggestions
-            for (int i = 0; i < suggestion.getNumFound(); i++) {
-              String alternative = suggestion.getAlternatives().get(i);
-              suggested.add(alternative);
-              SuggestWord sug = suggestedVsWord.get(alternative);
-              if (sug == null)  {
-                sug = new SuggestWord();
-                suggestedVsWord.put(alternative, sug);
-              }
-              sug.string = alternative;
-              // alternative frequency is present only for extendedResults=true
-              if (suggestion.getAlternativeFrequencies() != null && suggestion.getAlternativeFrequencies().size() > 0) {
-                Integer freq = suggestion.getAlternativeFrequencies().get(i);
-                if (freq != null) sug.freq += freq;
-              }
-            }
-          }
-          NamedList suggestions = (NamedList) nl.get("suggestions");
-          if(suggestions != null) {
-	      		List<Object> collationList = suggestions.getAll("collation");
-	      		List<Object> collationRankList = suggestions.getAll("collationInternalRank");
-	      		int i=0;
-	      		if(collationList != null) {
-		      		for(Object o : collationList)
-		      		{
-		      			if(o instanceof String)
-		      			{
-		      				SpellCheckCollation coll = new SpellCheckCollation();
-		      				coll.setCollationQuery((String) o);
-		      				if(collationRankList!= null && collationRankList.size()>0)
-		      				{
-			      				coll.setInternalRank((Integer) collationRankList.get(i));
-			      				i++;
-		      				}
-		      				SpellCheckCollation priorColl = collations.get(coll.getCollationQuery());
-		      				if(priorColl != null)
-		      				{
-		      					coll.setInternalRank(Math.max(coll.getInternalRank(),priorColl.getInternalRank()));
-		      				}
-		      				collations.put(coll.getCollationQuery(), coll);
-		      			} else
-		      			{
-		      				NamedList expandedCollation = (NamedList) o;		      				
-		      				SpellCheckCollation coll = new SpellCheckCollation();
-		      				coll.setCollationQuery((String) expandedCollation.get("collationQuery"));
-		      				coll.setHits((Integer) expandedCollation.get("hits"));
-		      				if(maxCollationTries>0)
-		      				{
-		      					coll.setInternalRank((Integer) expandedCollation.get("collationInternalRank"));
-		      				}
-		      				coll.setMisspellingsAndCorrections((NamedList) expandedCollation.get("misspellingsAndCorrections"));
-		      				SpellCheckCollation priorColl = collations.get(coll.getCollationQuery());
-		      				if(priorColl != null)
-		      				{
-		      					coll.setHits(coll.getHits() + priorColl.getHits());
-		      					coll.setInternalRank(Math.max(coll.getInternalRank(),priorColl.getInternalRank()));
-		      				}
-		      				collations.put(coll.getCollationQuery(), coll);
-		      			}
-		      		}
-	      		}
-          }
+        	mergeData.totalNumberShardResponses++;
+        	collectShardSuggestions(nl, mergeData);          
+          collectShardCollations(mergeData, nl, maxCollationTries);
         }
       }
     }
 
     // all shard responses have been collected
     // create token and get top suggestions
-    SpellingResult result = new SpellingResult(tokens); //todo: investigate, why does it need tokens beforehand?
-    for (Map.Entry<String, HashSet<String>> entry : origVsSuggested.entrySet()) {
-      String original = entry.getKey();
-      
-      //Only use this suggestion if all shards reported it as misspelled.
-      Integer numShards = origVsShards.get(original);
-      if(numShards<totalNumberShardResponses) {
-      	continue;
-      }
-      
-      HashSet<String> suggested = entry.getValue();
-      SuggestWordQueue sugQueue = new SuggestWordQueue(numSug);
-      for (String suggestion : suggested) {
-        SuggestWord sug = suggestedVsWord.get(suggestion);
-        sug.score = sd.getDistance(original, sug.string);
-        if (sug.score < min) continue;
-        sugQueue.insertWithOverflow(sug);
-        if (sugQueue.size() == numSug) {
-          // if queue full, maintain the minScore score
-          min = sugQueue.top().score;
-        }
-      }
-
-      // create token
-      SpellCheckResponse.Suggestion suggestion = origVsSuggestion.get(original);
-      Token token = new Token(original, suggestion.getStartOffset(), suggestion.getEndOffset());
-
-      // get top 'count' suggestions out of 'sugQueue.size()' candidates
-      SuggestWord[] suggestions = new SuggestWord[Math.min(count, sugQueue.size())];
-      // skip the first sugQueue.size() - count elements
-      for (int k=0; k < sugQueue.size() - count; k++) sugQueue.pop();
-      // now collect the top 'count' responses
-      for (int k = Math.min(count, sugQueue.size()) - 1; k >= 0; k--)  {
-        suggestions[k] = sugQueue.pop();
-      }
-
-      if (extendedResults) {
-        Integer o = origVsFreq.get(original);
-        if (o != null) result.addFrequency(token, o);
-        for (SuggestWord word : suggestions)
-          result.add(token, word.string, word.freq);
-      } else {
-        List<String> words = new ArrayList<String>(sugQueue.size());
-        for (SuggestWord word : suggestions) words.add(word.string);
-        result.add(token, words);
-      }
-    }
+    SolrSpellChecker checker = getSpellChecker(rb.req.getParams());    
+    SpellingResult result = checker.mergeSuggestions(mergeData, numSug, count, extendedResults);
     
     NamedList response = new SimpleOrderedMap();
 		NamedList suggestions = toNamedList(false, result, origQuery, extendedResults, collate);
 		if (collate) {
-			SpellCheckCollation[] sortedCollations = collations.values().toArray(new SpellCheckCollation[collations.size()]);
+			SpellCheckCollation[] sortedCollations = mergeData.collations.values().toArray(new SpellCheckCollation[mergeData.collations.size()]);
 			Arrays.sort(sortedCollations);
 			int i = 0;
 			while (i < maxCollations && i < sortedCollations.length) {
@@ -470,6 +308,101 @@ public class SpellCheckComponent extends
     response.add("suggestions", suggestions);
     rb.rsp.add("spellcheck", response);
   }
+    
+  @SuppressWarnings("unchecked")
+  private void collectShardSuggestions(NamedList nl, SpellCheckMergeData mergeData) {
+    SpellCheckResponse spellCheckResp = new SpellCheckResponse(nl);
+    for (SpellCheckResponse.Suggestion suggestion : spellCheckResp.getSuggestions()) {
+      mergeData.origVsSuggestion.put(suggestion.getToken(), suggestion);
+      HashSet<String> suggested = mergeData.origVsSuggested.get(suggestion.getToken());
+      if (suggested == null) {
+        suggested = new HashSet<String>();
+        mergeData.origVsSuggested.put(suggestion.getToken(), suggested);
+      }
+
+      // sum up original frequency          
+      int origFreq = 0;
+      Integer o = mergeData.origVsFreq.get(suggestion.getToken());
+      if (o != null)  origFreq += o;
+      origFreq += suggestion.getOriginalFrequency();
+      mergeData.origVsFreq.put(suggestion.getToken(), origFreq);
+      
+      //# shards reporting
+      Integer origShards = mergeData.origVsShards.get(suggestion.getToken());
+      if(origShards==null) {
+        mergeData.origVsShards.put(suggestion.getToken(), 1);
+      } else {
+        mergeData.origVsShards.put(suggestion.getToken(), ++origShards);
+      }            
+
+      // find best suggestions
+      for (int i = 0; i < suggestion.getNumFound(); i++) {
+        String alternative = suggestion.getAlternatives().get(i);
+        suggested.add(alternative);
+        SuggestWord sug = mergeData.suggestedVsWord.get(alternative);
+        if (sug == null)  {
+          sug = new SuggestWord();
+          mergeData.suggestedVsWord.put(alternative, sug);
+        }
+        sug.string = alternative;
+        // alternative frequency is present only for extendedResults=true
+        if (suggestion.getAlternativeFrequencies() != null && suggestion.getAlternativeFrequencies().size() > 0) {
+          Integer freq = suggestion.getAlternativeFrequencies().get(i);
+          if (freq != null) sug.freq += freq;
+        }
+      }
+    }
+  }
+  
+  @SuppressWarnings("unchecked")
+  private void collectShardCollations(SpellCheckMergeData mergeData, NamedList spellCheckResponse, int maxCollationTries) {
+    Map<String, SpellCheckCollation> collations = mergeData.collations;
+    NamedList suggestions = (NamedList) spellCheckResponse.get("suggestions");
+    if(suggestions != null) {
+      List<Object> collationList = suggestions.getAll("collation");
+      List<Object> collationRankList = suggestions.getAll("collationInternalRank");
+      int i=0;
+      if(collationList != null) {
+        for(Object o : collationList)
+        {
+          if(o instanceof String)
+          {
+            SpellCheckCollation coll = new SpellCheckCollation();
+            coll.setCollationQuery((String) o);
+            if(collationRankList!= null && collationRankList.size()>0)
+            {
+              coll.setInternalRank((Integer) collationRankList.get(i));
+              i++;
+            }
+            SpellCheckCollation priorColl = collations.get(coll.getCollationQuery());
+            if(priorColl != null)
+            {
+              coll.setInternalRank(Math.max(coll.getInternalRank(),priorColl.getInternalRank()));
+            }
+            collations.put(coll.getCollationQuery(), coll);
+          } else
+          {
+            NamedList expandedCollation = (NamedList) o;                  
+            SpellCheckCollation coll = new SpellCheckCollation();
+            coll.setCollationQuery((String) expandedCollation.get("collationQuery"));
+            coll.setHits((Integer) expandedCollation.get("hits"));
+            if(maxCollationTries>0)
+            {
+              coll.setInternalRank((Integer) expandedCollation.get("collationInternalRank"));
+            }
+            coll.setMisspellingsAndCorrections((NamedList) expandedCollation.get("misspellingsAndCorrections"));
+            SpellCheckCollation priorColl = collations.get(coll.getCollationQuery());
+            if(priorColl != null)
+            {
+              coll.setHits(coll.getHits() + priorColl.getHits());
+              coll.setInternalRank(Math.max(coll.getInternalRank(),priorColl.getInternalRank()));
+            }
+            collations.put(coll.getCollationQuery(), coll);
+          }
+        }
+      }
+    }
+  }
 
   private Collection<Token> getTokens(String q, Analyzer analyzer) throws IOException {
     Collection<Token> result = new ArrayList<Token>();

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ByteField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ByteField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ByteField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ByteField.java Fri Nov 11 13:08:38 2011
@@ -20,8 +20,6 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.ByteFieldSource;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.cache.ByteValuesCreator;
-import org.apache.lucene.search.cache.CachedArrayCreator;
 
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
@@ -48,7 +46,7 @@ public class ByteField extends FieldType
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    return new ByteFieldSource( new ByteValuesCreator( field.name, null, CachedArrayCreator.CACHE_VALUES_AND_BITS ) );
+    return new ByteFieldSource(field.name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/DoubleField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/DoubleField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/DoubleField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/DoubleField.java Fri Nov 11 13:08:38 2011
@@ -21,8 +21,6 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.DoubleValuesCreator;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 
@@ -48,7 +46,7 @@ public class DoubleField extends FieldTy
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    return new DoubleFieldSource( new DoubleValuesCreator( field.name, null, CachedArrayCreator.CACHE_VALUES_AND_BITS ) );
+    return new DoubleFieldSource(field.name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/FloatField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/FloatField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/FloatField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/FloatField.java Fri Nov 11 13:08:38 2011
@@ -20,8 +20,6 @@ package org.apache.solr.schema;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.FloatValuesCreator;
 import org.apache.solr.search.QParser;
 import org.apache.lucene.index.IndexableField;
 import org.apache.solr.response.TextResponseWriter;
@@ -46,7 +44,7 @@ public class FloatField extends FieldTyp
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    return new FloatFieldSource( new FloatValuesCreator( field.name, null, CachedArrayCreator.CACHE_VALUES_AND_BITS ) );
+    return new FloatFieldSource(field.name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/IntField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/IntField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/IntField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/IntField.java Fri Nov 11 13:08:38 2011
@@ -20,8 +20,6 @@ package org.apache.solr.schema;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.IntValuesCreator;
 import org.apache.solr.search.QParser;
 import org.apache.lucene.index.IndexableField;
 import org.apache.solr.response.TextResponseWriter;
@@ -46,7 +44,7 @@ public class IntField extends FieldType 
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    return new IntFieldSource(new IntValuesCreator( field.name, null, CachedArrayCreator.CACHE_VALUES_AND_BITS ) );
+    return new IntFieldSource(field.name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/LongField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/LongField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/LongField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/LongField.java Fri Nov 11 13:08:38 2011
@@ -21,8 +21,6 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.LongValuesCreator;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 
@@ -48,7 +46,7 @@ public class LongField extends FieldType
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    return new LongFieldSource( new LongValuesCreator( field.name, null, CachedArrayCreator.CACHE_VALUES_AND_BITS ) );
+    return new LongFieldSource(field.name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ShortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ShortField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ShortField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/ShortField.java Fri Nov 11 13:08:38 2011
@@ -20,8 +20,6 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.ShortFieldSource;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.ShortValuesCreator;
 
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
@@ -51,7 +49,7 @@ public class ShortField extends FieldTyp
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    return new ShortFieldSource(new ShortValuesCreator( field.name, null, CachedArrayCreator.CACHE_VALUES_AND_BITS ) );
+    return new ShortFieldSource(field.name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/TrieField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/TrieField.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/TrieField.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/schema/TrieField.java Fri Nov 11 13:08:38 2011
@@ -20,16 +20,12 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.NumericField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.search.*;
-import org.apache.lucene.search.cache.CachedArrayCreator;
-import org.apache.lucene.search.cache.DoubleValuesCreator;
+
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
 import org.apache.lucene.queries.function.valuesource.IntFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
-import org.apache.lucene.search.cache.FloatValuesCreator;
-import org.apache.lucene.search.cache.IntValuesCreator;
-import org.apache.lucene.search.cache.LongValuesCreator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.NumericUtils;
@@ -137,7 +133,6 @@ public class TrieField extends org.apach
   public SortField getSortField(SchemaField field, boolean top) {
     field.checkSortability();
 
-    int flags = CachedArrayCreator.CACHE_VALUES_AND_BITS;
     Object missingValue = null;
     boolean sortMissingLast  = field.sortMissingLast();
     boolean sortMissingFirst = field.sortMissingFirst();
@@ -150,8 +145,7 @@ public class TrieField extends org.apach
         else if( sortMissingFirst ) {
           missingValue = top ? Integer.MAX_VALUE : Integer.MIN_VALUE;
         }
-        return new SortField( new IntValuesCreator( field.getName(), 
-            FieldCache.NUMERIC_UTILS_INT_PARSER, flags ), top).setMissingValue( missingValue );
+        return new SortField( field.getName(), FieldCache.NUMERIC_UTILS_INT_PARSER, top).setMissingValue(missingValue);
       
       case FLOAT:
         if( sortMissingLast ) {
@@ -160,8 +154,7 @@ public class TrieField extends org.apach
         else if( sortMissingFirst ) {
           missingValue = top ? Float.POSITIVE_INFINITY : Float.NEGATIVE_INFINITY;
         }
-        return new SortField( new FloatValuesCreator( field.getName(), 
-            FieldCache.NUMERIC_UTILS_FLOAT_PARSER, flags ), top).setMissingValue( missingValue );
+        return new SortField( field.getName(), FieldCache.NUMERIC_UTILS_FLOAT_PARSER, top).setMissingValue(missingValue);
       
       case DATE: // fallthrough
       case LONG:
@@ -171,8 +164,7 @@ public class TrieField extends org.apach
         else if( sortMissingFirst ) {
           missingValue = top ? Long.MAX_VALUE : Long.MIN_VALUE;
         }
-        return new SortField( new LongValuesCreator( field.getName(), 
-            FieldCache.NUMERIC_UTILS_LONG_PARSER, flags ), top).setMissingValue( missingValue );
+        return new SortField( field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER, top).setMissingValue(missingValue);
         
       case DOUBLE:
         if( sortMissingLast ) {
@@ -181,8 +173,7 @@ public class TrieField extends org.apach
         else if( sortMissingFirst ) {
           missingValue = top ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
         }
-        return new SortField( new DoubleValuesCreator( field.getName(), 
-            FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, flags ), top).setMissingValue( missingValue );
+        return new SortField( field.getName(), FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, top).setMissingValue(missingValue);
         
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + field.name);
@@ -192,18 +183,17 @@ public class TrieField extends org.apach
   @Override
   public ValueSource getValueSource(SchemaField field, QParser qparser) {
     field.checkFieldCacheSource(qparser);
-    int flags = CachedArrayCreator.CACHE_VALUES_AND_BITS;
     switch (type) {
       case INTEGER:
-        return new IntFieldSource( new IntValuesCreator( field.getName(), FieldCache.NUMERIC_UTILS_INT_PARSER, flags ) );
+        return new IntFieldSource( field.getName(), FieldCache.NUMERIC_UTILS_INT_PARSER );
       case FLOAT:
-        return new FloatFieldSource( new FloatValuesCreator( field.getName(), FieldCache.NUMERIC_UTILS_FLOAT_PARSER, flags ));
+        return new FloatFieldSource( field.getName(), FieldCache.NUMERIC_UTILS_FLOAT_PARSER );
       case DATE:
-        return new TrieDateFieldSource( new LongValuesCreator( field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER, flags ));        
+        return new TrieDateFieldSource( field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER );        
       case LONG:
-        return new LongFieldSource( new LongValuesCreator( field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER, flags ) );
+        return new LongFieldSource( field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER );
       case DOUBLE:
-        return new DoubleFieldSource( new DoubleValuesCreator( field.getName(), FieldCache.NUMERIC_UTILS_DOUBLE_PARSER, flags ));
+        return new DoubleFieldSource( field.getName(), FieldCache.NUMERIC_UTILS_DOUBLE_PARSER );
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown type for trie field: " + field.name);
     }
@@ -573,8 +563,8 @@ public class TrieField extends org.apach
 
 class TrieDateFieldSource extends LongFieldSource {
 
-  public TrieDateFieldSource(LongValuesCreator creator) {
-    super(creator);
+  public TrieDateFieldSource(String field, FieldCache.LongParser parser) {
+    super(field, parser);
   }
 
   @Override

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java Fri Nov 11 13:08:38 2011
@@ -141,13 +141,7 @@ public abstract class AbstractLuceneSpel
   
   @Override
   public SpellingResult getSuggestions(SpellingOptions options) throws IOException {
-  	boolean shardRequest = false;
-  	SolrParams params = options.customParams;
-  	if(params!=null)
-  	{
-  		shardRequest = "true".equals(params.get(ShardParams.IS_SHARD));
-  	}
-    SpellingResult result = new SpellingResult(options.tokens);
+  	SpellingResult result = new SpellingResult(options.tokens);
     IndexReader reader = determineReader(options.reader);
     Term term = field != null ? new Term(field, "") : null;
     float theAccuracy = (options.accuracy == Float.MIN_VALUE) ? spellChecker.getAccuracy() : options.accuracy;
@@ -176,7 +170,7 @@ public abstract class AbstractLuceneSpel
 	          term = new Term(field, suggestions[i]);
 	          result.add(token, suggestions[i], reader.docFreq(term));
 	        }
-        } else if(shardRequest) {
+        } else {
         	List<String> suggList = Collections.emptyList();
         	result.add(token, suggList);
         }
@@ -187,7 +181,7 @@ public abstract class AbstractLuceneSpel
             suggList = suggList.subList(0, options.count);
           }
           result.add(token, suggList);
-        } else if(shardRequest) {
+        } else {
         	List<String> suggList = Collections.emptyList();
         	result.add(token, suggList);
         }
@@ -222,6 +216,7 @@ public abstract class AbstractLuceneSpel
   /*
   * @return the Accuracy used for the Spellchecker
   * */
+  @Override
   public float getAccuracy() {
     return accuracy;
   }
@@ -257,6 +252,7 @@ public abstract class AbstractLuceneSpel
     return sourceLocation;
   }
 
+  @Override
   public StringDistance getStringDistance() {
     return sd;
   }

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java Fri Nov 11 13:08:38 2011
@@ -18,7 +18,9 @@ package org.apache.solr.spelling;
  */
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Comparator;
+import java.util.List;
 
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
@@ -29,6 +31,8 @@ import org.apache.lucene.search.spell.Su
 import org.apache.lucene.search.spell.SuggestWord;
 import org.apache.lucene.search.spell.SuggestWordFrequencyComparator;
 import org.apache.lucene.search.spell.SuggestWordQueue;
+import org.apache.solr.common.params.ShardParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.SpellingParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
@@ -174,19 +178,41 @@ public class DirectSolrSpellChecker exte
   public SpellingResult getSuggestions(SpellingOptions options)
       throws IOException {
     LOG.debug("getSuggestions: " + options.tokens);
-    
+        
     SpellingResult result = new SpellingResult();
     float accuracy = (options.accuracy == Float.MIN_VALUE) ? checker.getAccuracy() : options.accuracy;
     SuggestMode mode = options.onlyMorePopular ? SuggestMode.SUGGEST_MORE_POPULAR : SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX;
     for (Token token : options.tokens) {
     	Term term = new Term(field, token.toString());
       SuggestWord[] suggestions = checker.suggestSimilar(term, 
-          options.count, options.reader, mode, accuracy);
-      result.addFrequency(token, options.reader.docFreq(term));
-      for (SuggestWord suggestion : suggestions) {
-        result.add(token, suggestion.string, suggestion.freq);      	
+          options.count, options.reader, mode, accuracy); 
+      
+      int docFreq = 0;
+      if(options.extendedResults || suggestions.length==0) {
+        docFreq = options.reader.docFreq(term);
+      }
+      
+      if(options.extendedResults) {        
+        result.addFrequency(token, docFreq);
+      }
+      if(suggestions.length==0 && docFreq==0) {
+        List<String> empty = Collections.emptyList();
+        result.add(token, empty);
+      } else {        
+        for (SuggestWord suggestion : suggestions) {
+          result.add(token, suggestion.string, suggestion.freq);      	
+        }
       }
     }
     return result;
   }
+  
+  @Override
+  public float getAccuracy() {
+    return checker.getAccuracy();
+  }
+  @Override
+  public StringDistance getStringDistance() {
+    return checker.getDistance();
+  }
 }

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java Fri Nov 11 13:08:38 2011
@@ -59,6 +59,9 @@ public class PossibilityIterator impleme
 	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();

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java Fri Nov 11 13:08:38 2011
@@ -17,13 +17,25 @@ package org.apache.solr.spelling;
  */
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.search.spell.LevensteinDistance;
+import org.apache.lucene.search.spell.StringDistance;
+import org.apache.lucene.search.spell.SuggestWord;
+import org.apache.lucene.search.spell.SuggestWordQueue;
+import org.apache.solr.client.solrj.response.SpellCheckResponse;
+import org.apache.solr.common.params.SpellingParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.component.SpellCheckMergeData;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.search.SolrIndexSearcher;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
 
 
 /**
@@ -64,6 +76,73 @@ public abstract class SolrSpellChecker {
     }
     return name;
   }
+  /**
+   * Integrate spelling suggestions from the various shards in a distributed environment.
+   * 
+   * @param mergeData
+   * @param numSug
+   * @param count
+   * @param extendedResults
+   */
+  public SpellingResult mergeSuggestions(SpellCheckMergeData mergeData, int numSug, int count, boolean extendedResults) {
+    float min = 0.5f;
+    try {
+      min = getAccuracy();
+    } catch(UnsupportedOperationException uoe) {
+      //just use .5 as a default
+    }
+    
+    StringDistance sd = getStringDistance() == null ? new LevensteinDistance() : getStringDistance();    
+    
+    SpellingResult result = new SpellingResult();
+    for (Map.Entry<String, HashSet<String>> entry : mergeData.origVsSuggested.entrySet()) {
+      String original = entry.getKey();
+      
+      //Only use this suggestion if all shards reported it as misspelled.
+      Integer numShards = mergeData.origVsShards.get(original);
+      if(numShards<mergeData.totalNumberShardResponses) {
+        continue;
+      }
+      
+      HashSet<String> suggested = entry.getValue();
+      SuggestWordQueue sugQueue = new SuggestWordQueue(numSug);
+      for (String suggestion : suggested) {
+        SuggestWord sug = mergeData.suggestedVsWord.get(suggestion);
+        sug.score = sd.getDistance(original, sug.string);
+        if (sug.score < min) continue;
+        sugQueue.insertWithOverflow(sug);
+        if (sugQueue.size() == numSug) {
+          // if queue full, maintain the minScore score
+          min = sugQueue.top().score;
+        }
+      }
+
+      // create token
+      SpellCheckResponse.Suggestion suggestion = mergeData.origVsSuggestion.get(original);
+      Token token = new Token(original, suggestion.getStartOffset(), suggestion.getEndOffset());
+
+      // get top 'count' suggestions out of 'sugQueue.size()' candidates
+      SuggestWord[] suggestions = new SuggestWord[Math.min(count, sugQueue.size())];
+      // skip the first sugQueue.size() - count elements
+      for (int k=0; k < sugQueue.size() - count; k++) sugQueue.pop();
+      // now collect the top 'count' responses
+      for (int k = Math.min(count, sugQueue.size()) - 1; k >= 0; k--)  {
+        suggestions[k] = sugQueue.pop();
+      }
+
+      if (extendedResults) {
+        Integer o = mergeData.origVsFreq.get(original);
+        if (o != null) result.addFrequency(token, o);
+        for (SuggestWord word : suggestions)
+          result.add(token, word.string, word.freq);
+      } else {
+        List<String> words = new ArrayList<String>(sugQueue.size());
+        for (SuggestWord word : suggestions) words.add(word.string);
+        result.add(token, words);
+      }
+    }
+    return result;
+  }
   
   public Analyzer getQueryAnalyzer() {
     return analyzer;
@@ -84,6 +163,21 @@ public abstract class SolrSpellChecker {
    * (re)Builds the spelling index.  May be a NOOP if the implementation doesn't require building, or can't be rebuilt.
    */
   public abstract void build(SolrCore core, SolrIndexSearcher searcher);
+  
+  /**
+   * Get the value of {@link SpellingParams#SPELLCHECK_ACCURACY} if supported.  
+   * Otherwise throws UnsupportedOperationException.
+   */
+  protected float getAccuracy() {
+    throw new UnsupportedOperationException();
+  }
+  
+  /**
+   * Get the distance implementation used by this spellchecker, or NULL if not applicable.
+   */
+  protected StringDistance getStringDistance()  {
+    throw new UnsupportedOperationException();
+  }
 
 
   /**

Modified: lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/update/FSUpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/update/FSUpdateLog.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/update/FSUpdateLog.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/java/org/apache/solr/update/FSUpdateLog.java Fri Nov 11 13:08:38 2011
@@ -355,7 +355,6 @@ class TransactionLog {
   OutputStream os;
   FastOutputStream fos;
   InputStream is;
-  long start;
 
   volatile boolean deleteOnClose = true;  // we can delete old tlogs since they are currently only used for real-time-get (and in the future, recovery)
 
@@ -417,7 +416,7 @@ class TransactionLog {
   public long writeData(Object o) {
     LogCodec codec = new LogCodec();
     try {
-      long pos = start + fos.size();   // if we had flushed, this should be equal to channel.position()
+      long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
       codec.init(fos);
       codec.writeVal(o);
       return pos;
@@ -430,7 +429,12 @@ class TransactionLog {
     try {
       this.tlogFile = tlogFile;
       raf = new RandomAccessFile(this.tlogFile, "rw");
-      start = raf.length();
+      long start = raf.length();
+      assert start==0;
+      if (start > 0) {
+        raf.setLength(0);
+        start = 0;
+      }
       // System.out.println("###start= "+start);
       channel = raf.getChannel();
       os = Channels.newOutputStream(channel);
@@ -474,12 +478,12 @@ class TransactionLog {
     LogCodec codec = new LogCodec();
     synchronized (fos) {
       try {
-        long pos = start + fos.size();   // if we had flushed, this should be equal to channel.position()
+        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
         SolrInputDocument sdoc = cmd.getSolrInputDocument();
 
         if (pos == 0) { // TODO: needs to be changed if we start writing a header first
           addGlobalStrings(sdoc.getFieldNames());
-          pos = start + fos.size();
+          pos = fos.size();
         }
 
         /***
@@ -496,6 +500,7 @@ class TransactionLog {
         codec.writeSolrInputDocument(cmd.getSolrInputDocument());
         // fos.flushBuffer();  // flush later
 
+        assert pos < fos.size();
         return pos;
       } catch (IOException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
@@ -507,10 +512,10 @@ class TransactionLog {
     LogCodec codec = new LogCodec();
     synchronized (fos) {
       try {
-        long pos = start + fos.size();   // if we had flushed, this should be equal to channel.position()
+        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
         if (pos == 0) {
           writeLogHeader(codec);
-          pos = start + fos.size();
+          pos = fos.size();
         }
         codec.init(fos);
         codec.writeTag(JavaBinCodec.ARR, 3);
@@ -519,6 +524,7 @@ class TransactionLog {
         BytesRef br = cmd.getIndexedId();
         codec.writeByteArray(br.bytes, br.offset, br.length);
         // fos.flushBuffer();  // flush later
+        assert pos < fos.size();
         return pos;
       } catch (IOException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
@@ -530,10 +536,10 @@ class TransactionLog {
     LogCodec codec = new LogCodec();
     synchronized (fos) {
       try {
-        long pos = start + fos.size();   // if we had flushed, this should be equal to channel.position()
+        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
         if (pos == 0) {
           writeLogHeader(codec);
-          pos = start + fos.size();
+          pos = fos.size();
         }
         codec.init(fos);
         codec.writeTag(JavaBinCodec.ARR, 3);
@@ -541,6 +547,7 @@ class TransactionLog {
         codec.writeLong(0);  // the version... should also just be one byte if 0
         codec.writeStr(cmd.query);
         // fos.flushBuffer();  // flush later
+        assert pos < fos.size();
         return pos;
       } catch (IOException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);

Modified: lucene/dev/branches/lucene2621/solr/core/src/test-files/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/test-files/solr/conf/solrconfig.xml?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/test-files/solr/conf/solrconfig.xml (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/test-files/solr/conf/solrconfig.xml Fri Nov 11 13:08:38 2011
@@ -324,6 +324,12 @@
       <str name="buildOnCommit">false</str>
     </lst>
     <lst name="spellchecker">
+      <str name="name">direct</str>
+      <str name="classname">DirectSolrSpellChecker</str>
+      <str name="field">lowerfilt</str>
+      <int name="minQueryLength">3</int>
+    </lst>
+    <lst name="spellchecker">
 			<str name="name">multipleFields</str>
 			<str name="field">lowerfilt1and2</str>
 			<str name="spellcheckIndexDir">spellcheckerMultipleFields</str>
@@ -398,6 +404,17 @@
       <str>spellcheck</str>
     </arr>
   </requestHandler>
+    <requestHandler name="spellCheckCompRH_Direct" class="org.apache.solr.handler.component.SearchHandler">
+    <lst name="defaults">
+      <str name="spellcheck.dictionary">direct</str>
+      <str name="spellcheck.onlyMorePopular">false</str>
+      <str name="spellcheck.extendedResults">false</str>
+      <str name="spellcheck.count">1</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/branches/lucene2621/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/handler/component/DistributedSpellCheckComponentTest.java Fri Nov 11 13:08:38 2011
@@ -30,10 +30,13 @@ import org.apache.solr.common.params.Mod
  */
 public class DistributedSpellCheckComponentTest extends BaseDistributedSearchTestCase {
   
+  private String requestHandlerName;
+  
 	public DistributedSpellCheckComponentTest()
 	{
 		//fixShardCount=true;
 		//shardCount=2;
+		//stress=0;
 	}
 	
   private String saveProp;
@@ -41,7 +44,8 @@ public class DistributedSpellCheckCompon
   public void setUp() throws Exception {
     // this test requires FSDir
     saveProp = System.getProperty("solr.directoryFactory");
-    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
+    System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");    
+    requestHandlerName = random.nextBoolean() ? "spellCheckCompRH" : "spellCheckCompRH_Direct";   
     super.setUp();
   }
   
@@ -104,15 +108,17 @@ public class DistributedSpellCheckCompon
     handle.put("maxScore", SKIPVAL);
     // we care only about the spellcheck results
     handle.put("response", SKIP);
+        
     q("q", "*:*", SpellCheckComponent.SPELLCHECK_BUILD, "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH");
     
-    query("q", "*:*", "fl", "id,lowerfilt", "spellcheck.q","toyata", "spellcheck", "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH");
-    query("q", "*:*", "fl", "id,lowerfilt", "spellcheck.q","toyata", "spellcheck", "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH", SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true");
-    query("q", "*:*", "fl", "id,lowerfilt", "spellcheck.q","bluo", "spellcheck", "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH", SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "4");
-    query("q", "The quick reb fox jumped over the lazy brown dogs", "fl", "id,lowerfilt", "spellcheck", "true", "qt", "spellCheckCompRH", "shards.qt", "spellCheckCompRH", SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "4", SpellCheckComponent.SPELLCHECK_COLLATE, "true");
+    query("q", "*:*", "fl", "id,lowerfilt", "spellcheck.q","toyata", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName);
+    query("q", "*:*", "fl", "id,lowerfilt", "spellcheck.q","toyata", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName, SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true");
+    query("q", "*:*", "fl", "id,lowerfilt", "spellcheck.q","bluo", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName, SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "4");
+    query("q", "The quick reb fox jumped over the lazy brown dogs", "fl", "id,lowerfilt", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName, SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "4", SpellCheckComponent.SPELLCHECK_COLLATE, "true");
 
-    query("q", "lowerfilt:(+quock +reb)", "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, "10", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true");
-    query("q", "lowerfilt:(+quock +reb)", "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, "10", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "false");
-    query("q", "lowerfilt:(+quock +reb)", "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, "0", SpellCheckComponent.SPELLCHECK_MAX_COLLATIONS, "1", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "false");
+    query("q", "lowerfilt:(+quock +reb)", "fl", "id,lowerfilt", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName, SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "10", SpellCheckComponent.SPELLCHECK_COLLATE, "true", SpellCheckComponent.SPELLCHECK_MAX_COLLATION_TRIES, "10", SpellCheckComponent.SPELLCHECK_MAX_COLLATIONS, "10", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "true");
+    query("q", "lowerfilt:(+quock +reb)", "fl", "id,lowerfilt", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName, SpellCheckComponent.SPELLCHECK_EXTENDED_RESULTS, "true", SpellCheckComponent.SPELLCHECK_COUNT, "10", SpellCheckComponent.SPELLCHECK_COLLATE, "true", SpellCheckComponent.SPELLCHECK_MAX_COLLATION_TRIES, "10", SpellCheckComponent.SPELLCHECK_MAX_COLLATIONS, "10", SpellCheckComponent.SPELLCHECK_COLLATE_EXTENDED_RESULTS, "false");
+    query("q", "lowerfilt:(+quock +reb)", "fl", "id,lowerfilt", "spellcheck", "true", "qt", requestHandlerName, "shards.qt", requestHandlerName, 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, "false");
+  
   }
 }

Modified: lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/FileBasedSpellCheckerTest.java Fri Nov 11 13:08:38 2011
@@ -175,7 +175,7 @@ public class FileBasedSpellCheckerTest e
     result = checker.getSuggestions(spellOpts);
     assertTrue("result is null and it shouldn't be", result != null);
     suggestions = result.get(spellOpts.tokens.iterator().next());
-    assertTrue("suggestions is not null and it should be", suggestions == null);
+    assertTrue("suggestions size should be 0", suggestions.size()==0);
     searcher.decref();
   }
 }

Modified: lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java?rev=1200865&r1=1200864&r2=1200865&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java (original)
+++ lucene/dev/branches/lucene2621/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java Fri Nov 11 13:08:38 2011
@@ -140,7 +140,7 @@ public class IndexBasedSpellCheckerTest 
     result = checker.getSuggestions(spellOpts);
     assertTrue("result is null and it shouldn't be", result != null);
     suggestions = result.get(spellOpts.tokens.iterator().next());
-    assertTrue("suggestions is not null and it should be", suggestions == null);
+    assertTrue("suggestions size should be 0", suggestions.size()==0);
 
     //test something that is spelled correctly
     spellOpts.tokens = queryConverter.convert("document");
@@ -215,7 +215,7 @@ public class IndexBasedSpellCheckerTest 
     result = checker.getSuggestions(spellOpts);
     assertTrue("result is null and it shouldn't be", result != null);
     suggestions = result.get(spellOpts.tokens.iterator().next());
-    assertTrue("suggestions is not null and it should be", suggestions == null);
+    assertTrue("suggestions size should be 0", suggestions.size()==0);
 
     spellOpts.tokens = queryConverter.convert("document");
     result = checker.getSuggestions(spellOpts);
@@ -328,7 +328,7 @@ public class IndexBasedSpellCheckerTest 
     result = checker.getSuggestions(spellOpts);
     assertTrue("result is null and it shouldn't be", result != null);
     suggestions = result.get(spellOpts.tokens.iterator().next());
-    assertTrue("suggestions is not null and it should be", suggestions == null);
+    assertTrue("suggestions size should be 0", suggestions.size()==0);
 
     spellOpts.tokens = queryConverter.convert("Caroline");
     result = checker.getSuggestions(spellOpts);