You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/08/13 15:53:27 UTC

svn commit: r1372423 [42/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ d...

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/component/TermVectorComponent.java Mon Aug 13 13:52:46 2012
@@ -11,11 +11,11 @@ import java.util.Set;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Terms;
@@ -141,6 +141,15 @@ public class TermVectorComponent extends
 
     NamedList<Object> termVectors = new NamedList<Object>();
     rb.rsp.add(TERM_VECTORS, termVectors);
+
+    IndexSchema schema = rb.req.getSchema();
+    SchemaField keyField = schema.getUniqueKeyField();
+    String uniqFieldName = null;
+    if (keyField != null) {
+      uniqFieldName = keyField.getName();
+      termVectors.add("uniqueKeyFieldName", uniqFieldName);
+    }
+
     FieldOptions allFields = new FieldOptions();
     //figure out what options we have, and try to get the appropriate vector
     allFields.termFreq = params.getBool(TermVectorParams.TF, false);
@@ -158,8 +167,6 @@ public class TermVectorComponent extends
       allFields.tfIdf = true;
     }
 
-    //use this to validate our fields
-    IndexSchema schema = rb.req.getSchema();
     //Build up our per field mapping
     Map<String, FieldOptions> fieldOptions = new HashMap<String, FieldOptions>();
     NamedList<List<String>> warnings = new NamedList<List<String>>();
@@ -175,6 +182,11 @@ public class TermVectorComponent extends
         // workarround SOLR-3523
         if (null == field || "score".equals(field)) continue; 
 
+        // we don't want to issue warnings about the uniqueKey field
+        // since it can cause lots of confusion in distributed requests
+        // where the uniqueKey field is injected into the fl for merging
+        final boolean fieldIsUniqueKey = field.equals(uniqFieldName);
+
         SchemaField sf = schema.getFieldOrNull(field);
         if (sf != null) {
           if (sf.storeTermVector()) {
@@ -190,15 +202,15 @@ public class TermVectorComponent extends
             option.tfIdf = params.getFieldBool(field, TermVectorParams.TF_IDF, allFields.tfIdf);
             //Validate these are even an option
             option.positions = params.getFieldBool(field, TermVectorParams.POSITIONS, allFields.positions);
-            if (option.positions && !sf.storeTermPositions()){
+            if (option.positions && !sf.storeTermPositions() && !fieldIsUniqueKey){
               noPos.add(field);
             }
             option.offsets = params.getFieldBool(field, TermVectorParams.OFFSETS, allFields.offsets);
-            if (option.offsets && !sf.storeTermOffsets()){
+            if (option.offsets && !sf.storeTermOffsets() && !fieldIsUniqueKey){
               noOff.add(field);
             }
           } else {//field doesn't have term vectors
-            noTV.add(field);
+            if (!fieldIsUniqueKey) noTV.add(field);
           }
         } else {
           //field doesn't exist
@@ -206,6 +218,11 @@ public class TermVectorComponent extends
         }
       }
     } //else, deal with all fields
+
+    // NOTE: currently all typs of warnings are schema driven, and garunteed
+    // to be consistent across all shards - if additional types of warnings 
+    // are added that might be differnet between shards, finishStage() needs 
+    // to be changed to account for that.
     boolean hasWarnings = false;
     if (!noTV.isEmpty()) {
       warnings.add("noTermVectors", noTV);
@@ -236,11 +253,7 @@ public class TermVectorComponent extends
 
     IndexReader reader = searcher.getIndexReader();
     //the TVMapper is a TermVectorMapper which can be used to optimize loading of Term Vectors
-    SchemaField keyField = schema.getUniqueKeyField();
-    String uniqFieldName = null;
-    if (keyField != null) {
-      uniqFieldName = keyField.getName();
-    }
+
     //Only load the id field to get the uniqueKey of that
     //field
 
@@ -277,7 +290,6 @@ public class TermVectorComponent extends
     while (iter.hasNext()) {
       Integer docId = iter.next();
       NamedList<Object> docNL = new NamedList<Object>();
-      termVectors.add("doc-" + docId, docNL);
 
       if (keyField != null) {
         reader.document(docId, getUniqValue);
@@ -286,9 +298,13 @@ public class TermVectorComponent extends
           uniqVal = uniqValues.get(0);
           uniqValues.clear();
           docNL.add("uniqueKey", uniqVal);
-          termVectors.add("uniqueKeyFieldName", uniqFieldName);
+          termVectors.add(uniqVal, docNL);
         }
+      } else {
+        // support for schemas w/o a unique key,
+        termVectors.add("doc-" + docId, docNL);
       }
+
       if ( null != fields ) {
         for (Map.Entry<String, FieldOptions> entry : fieldOptions.entrySet()) {
           final String field = entry.getKey();
@@ -301,10 +317,8 @@ public class TermVectorComponent extends
       } else {
         // extract all fields
         final Fields vectors = reader.getTermVectors(docId);
-        final FieldsEnum fieldsEnum = vectors.iterator();
-        String field;
-        while((field = fieldsEnum.next()) != null) {
-          Terms terms = fieldsEnum.terms();
+        for (String field : vectors) {
+          Terms terms = vectors.terms(field);
           if (terms != null) {
             termsEnum = terms.iterator(termsEnum);
             mapOneVector(docNL, allFields, reader, docId, termsEnum, field);
@@ -329,28 +343,19 @@ public class TermVectorComponent extends
         termInfo.add("tf", freq);
       }
 
-      dpEnum = termsEnum.docsAndPositions(null, dpEnum, fieldOptions.offsets);
-      boolean useOffsets = fieldOptions.offsets;
-      if (dpEnum == null) {
-        useOffsets = false;
-        dpEnum = termsEnum.docsAndPositions(null, dpEnum, false);
-      }
-
+      dpEnum = termsEnum.docsAndPositions(null, dpEnum);
+      boolean useOffsets = false;
       boolean usePositions = false;
       if (dpEnum != null) {
         dpEnum.nextDoc();
         usePositions = fieldOptions.positions;
-      }
-
-      NamedList<Number> theOffsets = null;
-      if (useOffsets) {
-        theOffsets = new NamedList<Number>();
-        termInfo.add("offsets", theOffsets);
+        useOffsets = fieldOptions.offsets;
       }
 
       NamedList<Integer> positionsNL = null;
+      NamedList<Number> theOffsets = null;
 
-      if (usePositions || theOffsets != null) {
+      if (usePositions || useOffsets) {
         for (int i = 0; i < freq; i++) {
           final int pos = dpEnum.nextPosition();
           if (usePositions && pos >= 0) {
@@ -361,6 +366,15 @@ public class TermVectorComponent extends
             positionsNL.add("position", pos);
           }
 
+          if (useOffsets && theOffsets == null) {
+            if (dpEnum.startOffset() == -1) {
+              useOffsets = false;
+            } else {
+              theOffsets = new NamedList<Number>();
+              termInfo.add("offsets", theOffsets);
+            }
+          }
+
           if (theOffsets != null) {
             theOffsets.add("start", dpEnum.startOffset());
             theOffsets.add("end", dpEnum.endOffset());
@@ -394,46 +408,6 @@ public class TermVectorComponent extends
     return result;
   }
 
-  @Override
-  public int distributedProcess(ResponseBuilder rb) throws IOException {
-    int result = ResponseBuilder.STAGE_DONE;
-    if (rb.stage == ResponseBuilder.STAGE_GET_FIELDS) {
-      //Go ask each shard for it's vectors
-      // for each shard, collect the documents for that shard.
-      HashMap<String, Collection<ShardDoc>> shardMap = new HashMap<String, Collection<ShardDoc>>();
-      for (ShardDoc sdoc : rb.resultIds.values()) {
-        Collection<ShardDoc> shardDocs = shardMap.get(sdoc.shard);
-        if (shardDocs == null) {
-          shardDocs = new ArrayList<ShardDoc>();
-          shardMap.put(sdoc.shard, shardDocs);
-        }
-        shardDocs.add(sdoc);
-      }
-      // Now create a request for each shard to retrieve the stored fields
-      for (Collection<ShardDoc> shardDocs : shardMap.values()) {
-        ShardRequest sreq = new ShardRequest();
-        sreq.purpose = ShardRequest.PURPOSE_GET_FIELDS;
-
-        sreq.shards = new String[]{shardDocs.iterator().next().shard};
-
-        sreq.params = new ModifiableSolrParams();
-
-        // add original params
-        sreq.params.add(rb.req.getParams());
-        sreq.params.remove(CommonParams.Q);//remove the query
-        ArrayList<String> ids = new ArrayList<String>(shardDocs.size());
-        for (ShardDoc shardDoc : shardDocs) {
-          ids.add(shardDoc.id.toString());
-        }
-        sreq.params.add(TermVectorParams.DOC_IDS, StrUtils.join(ids, ','));
-
-        rb.addRequest(this, sreq);
-      }
-      result = ResponseBuilder.STAGE_DONE;
-    }
-    return result;
-  }
-
   private static int getDocFreq(IndexReader reader, String field, BytesRef term) {
     int result = 1;
     try {
@@ -449,6 +423,40 @@ public class TermVectorComponent extends
 
   }
 
+  @Override
+  public void finishStage(ResponseBuilder rb) {
+    if (rb.stage == ResponseBuilder.STAGE_GET_FIELDS) {
+      
+      NamedList termVectors = new NamedList<Object>();
+      Map.Entry<String, Object>[] arr = new NamedList.NamedListEntry[rb.resultIds.size()];
+
+      for (ShardRequest sreq : rb.finished) {
+        if ((sreq.purpose & ShardRequest.PURPOSE_GET_FIELDS) == 0 || !sreq.params.getBool(COMPONENT_NAME, false)) {
+          continue;
+        }
+        for (ShardResponse srsp : sreq.responses) {
+          NamedList<Object> nl = (NamedList<Object>)srsp.getSolrResponse().getResponse().get(TERM_VECTORS);
+          for (int i=0; i < nl.size(); i++) {
+            String key = nl.getName(i);
+            ShardDoc sdoc = rb.resultIds.get(key);
+            if (null == sdoc) {
+              // metadata, only need from one node, leave in order
+              if (termVectors.indexOf(key,0) < 0) {
+                termVectors.add(key, nl.getVal(i));
+              }
+            } else {
+              int idx = sdoc.positionInResponse;
+              arr[idx] = new NamedList.NamedListEntry<Object>(key, nl.getVal(i));
+            }
+          }
+        }
+      }
+      // remove nulls in case not all docs were able to be retrieved
+      termVectors.addAll(SolrPluginUtils.removeNulls(new NamedList<Object>(arr)));
+      rb.rsp.add(TERM_VECTORS, termVectors);
+    }
+  }
+
   //////////////////////// NamedListInitializedPlugin methods //////////////////////
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java Mon Aug 13 13:52:46 2012
@@ -122,8 +122,8 @@ public class JavabinLoader extends Conte
       for (String s : update.getDeleteById()) {
         delcmd.id = s;
         processor.processDelete(delcmd);
+        delcmd.clear();
       }
-      delcmd.id = null;
     }
     
     if(update.getDeleteQuery() != null) {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/request/SimpleFacets.java Mon Aug 13 13:52:46 2012
@@ -397,7 +397,7 @@ public class SimpleFacets {
           Integer.MAX_VALUE,
           10, TimeUnit.SECONDS, // terminate idle threads after 10 sec
           new SynchronousQueue<Runnable>()  // directly hand off tasks
-          , new DefaultSolrThreadFactory("facetExectutor")
+          , new DefaultSolrThreadFactory("facetExecutor")
   );
   
   /**
@@ -756,7 +756,7 @@ public class SimpleFacets {
             // TODO: specialize when base docset is a bitset or hash set (skipDocs)?  or does it matter for this?
             // TODO: do this per-segment for better efficiency (MultiDocsEnum just uses base class impl)
             // TODO: would passing deleted docs lead to better efficiency over checking the fastForRandomSet?
-            docsEnum = termsEnum.docs(null, docsEnum, false);
+            docsEnum = termsEnum.docs(null, docsEnum, 0);
             c=0;
 
             if (docsEnum instanceof MultiDocsEnum) {
@@ -924,6 +924,11 @@ public class SimpleFacets {
               (SolrException.ErrorCode.BAD_REQUEST,
                   "date facet infinite loop (is gap negative?)");
         }
+        if (high.equals(low)) {
+          throw new SolrException
+            (SolrException.ErrorCode.BAD_REQUEST,
+             "date facet infinite loop: gap is effectively zero");
+        }
         final boolean includeLower =
             (include.contains(FacetRangeInclude.LOWER) ||
                 (include.contains(FacetRangeInclude.EDGE) && low.equals(start)));
@@ -1113,6 +1118,11 @@ public class SimpleFacets {
           (SolrException.ErrorCode.BAD_REQUEST,
            "range facet infinite loop (is gap negative? did the math overflow?)");
       }
+      if (high.compareTo(low) == 0) {
+        throw new SolrException
+          (SolrException.ErrorCode.BAD_REQUEST,
+           "range facet infinite loop: gap is either zero, or too small relative start/end and caused underflow: " + low + " + " + gap + " = " + high );
+      }
       
       final boolean includeLower = 
         (include.contains(FacetRangeInclude.LOWER) ||

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/CurrencyField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/CurrencyField.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/CurrencyField.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/CurrencyField.java Mon Aug 13 13:52:46 2012
@@ -136,9 +136,9 @@ public class CurrencyField extends Field
 
     IndexableField[] f = new IndexableField[field.stored() ? 3 : 2];
     SchemaField amountField = getAmountField(field);
-    f[0] = amountField.createField(String.valueOf(value.getAmount()), amountField.omitNorms() ? 1F : boost);
+    f[0] = amountField.createField(String.valueOf(value.getAmount()), amountField.indexed() && !amountField.omitNorms() ? boost : 1F);
     SchemaField currencyField = getCurrencyField(field);
-    f[1] = currencyField.createField(value.getCurrencyCode(), currencyField.omitNorms() ? 1F : boost);
+    f[1] = currencyField.createField(value.getCurrencyCode(), currencyField.indexed() && !currencyField.omitNorms() ? boost : 1F);
 
     if (field.stored()) {
       org.apache.lucene.document.FieldType customType = new org.apache.lucene.document.FieldType();
@@ -148,7 +148,7 @@ public class CurrencyField extends Field
       if (storedValue.indexOf(",") < 0) {
         storedValue += "," + defaultCurrency;
       }
-      f[2] = createField(field.getName(), storedValue, customType, boost);
+      f[2] = createField(field.getName(), storedValue, customType, 1F);
     }
 
     return f;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/FieldTypePluginLoader.java Mon Aug 13 13:52:46 2012
@@ -19,9 +19,9 @@ package org.apache.solr.schema;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.analysis.core.KeywordTokenizerFactory;
 import org.apache.lucene.analysis.util.*;
 import org.apache.lucene.util.Version;
-import org.apache.solr.analysis.KeywordTokenizerFactory;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.util.DOMUtil;
@@ -73,7 +73,7 @@ public final class FieldTypePluginLoader
 
 
   @Override
-  protected FieldType create( ResourceLoader loader, 
+  protected FieldType create( SolrResourceLoader loader, 
                               String name, 
                               String className, 
                               Node node ) throws Exception {
@@ -225,7 +225,30 @@ public final class FieldTypePluginLoader
     if (node == null) return null;
     NamedNodeMap attrs = node.getAttributes();
     String analyzerName = DOMUtil.getAttr(attrs,"class");
+
+    // check for all of these up front, so we can error if used in 
+    // conjunction with an explicit analyzer class.
+    NodeList charFilterNodes = (NodeList)xpath.evaluate
+      ("./charFilter",  node, XPathConstants.NODESET);
+    NodeList tokenizerNodes = (NodeList)xpath.evaluate
+      ("./tokenizer", node, XPathConstants.NODESET);
+    NodeList tokenFilterNodes = (NodeList)xpath.evaluate
+      ("./filter", node, XPathConstants.NODESET);
+      
     if (analyzerName != null) {
+
+      // explicitly check for child analysis factories instead of
+      // just any child nodes, because the user might have their
+      // own custom nodes (ie: <description> or something like that)
+      if (0 != charFilterNodes.getLength() ||
+          0 != tokenizerNodes.getLength() ||
+          0 != tokenFilterNodes.getLength()) {
+        throw new SolrException
+        ( SolrException.ErrorCode.SERVER_ERROR,
+          "Configuration Error: Analyzer class='" + analyzerName +
+          "' can not be combined with nested analysis factories");
+      }
+
       try {
         // No need to be core-aware as Analyzers are not in the core-aware list
         final Class<? extends Analyzer> clazz = loader.findClass(analyzerName, Analyzer.class);
@@ -286,8 +309,7 @@ public final class FieldTypePluginLoader
       }
     };
 
-    charFilterLoader.load( loader, (NodeList)xpath.evaluate("./charFilter",  node, XPathConstants.NODESET) );
-                            
+    charFilterLoader.load( loader, charFilterNodes );
 
     // Load the Tokenizer
     // Although an analyzer only allows a single Tokenizer, we load a list to make sure
@@ -319,13 +341,12 @@ public final class FieldTypePluginLoader
       }
     };
 
-    tokenizerLoader.load( loader, (NodeList)xpath.evaluate("./tokenizer", node, XPathConstants.NODESET) );
+    tokenizerLoader.load( loader, tokenizerNodes );
     
     // Make sure something was loaded
     if( tokenizers.isEmpty() ) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"analyzer without class or tokenizer & filter list");
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"analyzer without class or tokenizer");
     }
-    
 
     // Load the Filters
 
@@ -353,7 +374,7 @@ public final class FieldTypePluginLoader
         return null; // used for map registration
       }
     };
-    filterLoader.load( loader, (NodeList)xpath.evaluate("./filter", node, XPathConstants.NODESET) );
+    filterLoader.load( loader, tokenFilterNodes );
     
     return new TokenizerChain(charFilters.toArray(new CharFilterFactory[charFilters.size()]),
                               tokenizers.get(0), filters.toArray(new TokenFilterFactory[filters.size()]));

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Mon Aug 13 13:52:46 2012
@@ -40,6 +40,7 @@ import javax.xml.xpath.XPathConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -104,12 +105,16 @@ public final class IndexSchema {
       name = DEFAULT_SCHEMA_FILE;
     this.resourceName = name;
     loader = solrConfig.getResourceLoader();
-    if (is == null) {
-      is = new InputSource(loader.openSchema(name));
-      is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+    try {
+      if (is == null) {
+        is = new InputSource(loader.openSchema(name));
+        is.setSystemId(SystemIdResolver.createSystemIdFromResourceName(name));
+      }
+      readSchema(is);
+      loader.inform( loader );
+    } catch (IOException e) {
+      throw new RuntimeException(e);
     }
-    readSchema(is);
-    loader.inform( loader );
   }
   
   /**
@@ -494,7 +499,10 @@ public final class IndexSchema {
         log.error("uniqueKey is not stored - distributed search will not work");
       }
       if (uniqueKeyField.multiValued()) {
-        log.error("uniqueKey should not be multivalued");
+        String msg = "uniqueKey field ("+uniqueKeyFieldName+
+          ") can not be configured to be multivalued";
+        log.error(msg);
+        throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, msg );
       }
       uniqueKeyFieldName=uniqueKeyField.getName();
       uniqueKeyFieldType=uniqueKeyField.getType();
@@ -693,7 +701,7 @@ public final class IndexSchema {
     return newArr;
   }
 
-  static SimilarityFactory readSimilarity(ResourceLoader loader, Node node) {
+  static SimilarityFactory readSimilarity(SolrResourceLoader loader, Node node) {
     if (node==null) {
       return null;
     } else {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/LatLonType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/LatLonType.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/LatLonType.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/LatLonType.java Mon Aug 13 13:52:46 2012
@@ -76,18 +76,18 @@ public class LatLonType extends Abstract
       }
       //latitude
       SchemaField lat = subField(field, i);
-      f[i] = lat.createField(String.valueOf(latLon[LAT]), lat.omitNorms() ? 1F : boost);
+      f[i] = lat.createField(String.valueOf(latLon[LAT]), lat.indexed() && !lat.omitNorms() ? boost : 1f);
       i++;
       //longitude
       SchemaField lon = subField(field, i);
-      f[i] = lon.createField(String.valueOf(latLon[LON]), lon.omitNorms() ? 1F : boost);
+      f[i] = lon.createField(String.valueOf(latLon[LON]), lon.indexed() && !lon.omitNorms() ? boost : 1f);
 
     }
 
     if (field.stored()) {
       FieldType customType = new FieldType();
       customType.setStored(true);
-      f[f.length - 1] = createField(field.getName(), externalVal, customType, boost);
+      f[f.length - 1] = createField(field.getName(), externalVal, customType, 1f);
     }
     return f;
   }
@@ -350,13 +350,13 @@ class SpatialDistanceQuery extends Exten
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
       return new SpatialScorer(context, acceptDocs, this, queryWeight);
     }
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      return ((SpatialScorer)scorer(context, true, true, FeatureFlags.DOCS, context.reader().getLiveDocs())).explain(doc);
+      return ((SpatialScorer)scorer(context, true, true, PostingFeatures.DOCS_AND_FREQS, context.reader().getLiveDocs())).explain(doc);
     }
   }
 

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/PointType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/PointType.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/PointType.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/schema/PointType.java Mon Aug 13 13:52:46 2012
@@ -83,7 +83,8 @@ public class PointType extends Coordinat
 
     if (field.indexed()) {
       for (int i=0; i<dimension; i++) {
-        f[i] = subField(field, i).createField(point[i], boost);
+        SchemaField sf = subField(field, i);
+        f[i] = sf.createField(point[i], sf.indexed() && !sf.omitNorms() ? boost : 1f);
       }
     }
 
@@ -91,7 +92,7 @@ public class PointType extends Coordinat
       String storedVal = externalVal;  // normalize or not?
       FieldType customType = new FieldType();
       customType.setStored(true);
-      f[f.length - 1] = createField(field.getName(), storedVal, customType, boost);
+      f[f.length - 1] = createField(field.getName(), storedVal, customType, 1f);
     }
     
     return f;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParserPlugin.java Mon Aug 13 13:52:46 2012
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.StopFilterFactory;
 import org.apache.lucene.analysis.util.TokenFilterFactory;
 import org.apache.lucene.queries.function.BoostedQuery;
 import org.apache.lucene.queries.function.FunctionQuery;
@@ -36,7 +37,6 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.search.*;
-import org.apache.solr.analysis.StopFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.search.SolrQueryParser.MagicFieldName;
 import org.apache.solr.common.params.DisMaxParams;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/Grouping.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/Grouping.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/Grouping.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/Grouping.java Mon Aug 13 13:52:46 2012
@@ -787,7 +787,7 @@ public class Grouping {
           SchemaField schemaField = searcher.getSchema().getField(groupBy);
           FieldType fieldType = schemaField.getType();
           String readableValue = fieldType.indexedToReadable(group.groupValue.utf8ToString());
-          IndexableField field = schemaField.createField(readableValue, 0.0f);
+          IndexableField field = schemaField.createField(readableValue, 1.0f);
           nl.add("groupValue", fieldType.toObject(field));
         } else {
           nl.add("groupValue", null);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Mon Aug 13 13:52:46 2012
@@ -216,7 +216,7 @@ class JoinQuery extends Query {
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
       if (filter == null) {
         boolean debug = rb != null && rb.isDebug();
         long start = debug ? System.currentTimeMillis() : 0;
@@ -342,7 +342,7 @@ class JoinQuery extends Query {
         if (freq < minDocFreqFrom) {
           fromTermDirectCount++;
           // OK to skip liveDocs, since we check for intersection with docs matching query
-          fromDeState.docsEnum = fromDeState.termsEnum.docs(null, fromDeState.docsEnum, false);
+          fromDeState.docsEnum = fromDeState.termsEnum.docs(null, fromDeState.docsEnum, 0);
           DocsEnum docsEnum = fromDeState.docsEnum;
 
           if (docsEnum instanceof MultiDocsEnum) {
@@ -407,7 +407,7 @@ class JoinQuery extends Query {
               toTermDirectCount++;
 
               // need to use liveDocs here so we don't map to any deleted ones
-              toDeState.docsEnum = toDeState.termsEnum.docs(toDeState.liveDocs, toDeState.docsEnum, false);
+              toDeState.docsEnum = toDeState.termsEnum.docs(toDeState.liveDocs, toDeState.docsEnum, 0);
               DocsEnum docsEnum = toDeState.docsEnum;              
 
               if (docsEnum instanceof MultiDocsEnum) {
@@ -485,7 +485,7 @@ class JoinQuery extends Query {
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, true, false, FeatureFlags.DOCS, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, true, false, PostingFeatures.DOCS_AND_FREQS, context.reader().getLiveDocs());
       boolean exists = scorer.advance(doc) == doc;
 
       ComplexExplanation result = new ComplexExplanation();

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Mon Aug 13 13:52:46 2012
@@ -121,7 +121,7 @@ public class SolrConstantScoreQuery exte
 
     @Override
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, FeatureFlags flags, Bits acceptDocs) throws IOException {
+        boolean topScorer, PostingFeatures flags, Bits acceptDocs) throws IOException {
       return new ConstantScorer(context, this, queryWeight, acceptDocs);
     }
 

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Mon Aug 13 13:52:46 2012
@@ -35,7 +35,7 @@ import org.apache.lucene.document.Stored
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.*;
 import org.apache.lucene.search.*;
-import org.apache.lucene.search.Weight.FeatureFlags;
+import org.apache.lucene.search.Weight.PostingFeatures;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.NRTCachingDirectory;
@@ -599,7 +599,7 @@ public class SolrIndexSearcher extends I
     if (!termsEnum.seekExact(termBytes, false)) {
       return -1;
     }
-    DocsEnum docs = termsEnum.docs(atomicReader.getLiveDocs(), null, false);
+    DocsEnum docs = termsEnum.docs(atomicReader.getLiveDocs(), null, 0);
     if (docs == null) return -1;
     int id = docs.nextDoc();
     return id == DocIdSetIterator.NO_MORE_DOCS ? -1 : id;
@@ -621,7 +621,7 @@ public class SolrIndexSearcher extends I
 
       final Bits liveDocs = reader.getLiveDocs();
       
-      final DocsEnum docs = reader.termDocsEnum(liveDocs, field, idBytes, false);
+      final DocsEnum docs = reader.termDocsEnum(liveDocs, field, idBytes, 0);
 
       if (docs == null) continue;
       int id = docs.nextDoc();
@@ -926,7 +926,7 @@ public class SolrIndexSearcher extends I
     int bitsSet = 0;
     OpenBitSet obs = null;
 
-    DocsEnum docsEnum = deState.termsEnum.docs(deState.liveDocs, deState.docsEnum, false);
+    DocsEnum docsEnum = deState.termsEnum.docs(deState.liveDocs, deState.docsEnum, 0);
     if (deState.docsEnum == null) {
       deState.docsEnum = docsEnum;
     }
@@ -1004,7 +1004,7 @@ public class SolrIndexSearcher extends I
           if (terms != null) {
             final TermsEnum termsEnum = terms.iterator(null);
             if (termsEnum.seekExact(termBytes, false)) {
-              docsEnum = termsEnum.docs(liveDocs, null, false);
+              docsEnum = termsEnum.docs(liveDocs, null, 0);
             }
           }
 
@@ -2213,7 +2213,7 @@ class FilterImpl extends Filter {
         iterators.add(iter);
       }
       for (Weight w : weights) {
-        Scorer scorer = w.scorer(context, true, false, FeatureFlags.DOCS, context.reader().getLiveDocs());
+        Scorer scorer = w.scorer(context, true, false, PostingFeatures.DOCS_AND_FREQS, context.reader().getLiveDocs());
         if (scorer == null) return null;
         iterators.add(scorer);
       }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java Mon Aug 13 13:52:46 2012
@@ -107,6 +107,25 @@ public abstract class ValueSourceParser 
         return new LiteralValueSource(fp.parseArg());
       }
     });
+    addParser("threadid", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws ParseException {
+        return new LongConstValueSource(Thread.currentThread().getId());
+      }
+    });
+    addParser("sleep", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws ParseException {
+        int ms = fp.parseInt();
+        ValueSource source = fp.parseValueSource();
+        try {
+          Thread.sleep(ms);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+        return source;
+      }
+    });
     addParser("rord", new ValueSourceParser() {
       @Override
       public ValueSource parse(FunctionQParser fp) throws ParseException {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/function/FileFloatSource.java Mon Aug 13 13:52:46 2012
@@ -278,7 +278,7 @@ public class FileFloatSource extends Val
           continue;
         }
 
-        docsEnum = termsEnum.docs(null, docsEnum, false);
+        docsEnum = termsEnum.docs(null, docsEnum, 0);
         int doc;
         while ((doc = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
           vals[doc] = fval;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java Mon Aug 13 13:52:46 2012
@@ -110,9 +110,9 @@ public class SearchGroupsResultTransform
           if (sortValue instanceof BytesRef) {
             UnicodeUtil.UTF8toUTF16((BytesRef)sortValue, spare);
             String indexedValue = spare.toString();
-            sortValue = (Comparable) fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 0.0f));
+            sortValue = (Comparable) fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 1.0f));
           } else if (sortValue instanceof String) {
-            sortValue = (Comparable) fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 0.0f));
+            sortValue = (Comparable) fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 1.0f));
           }
         }
         convertedSortValues[i] = sortValue;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java Mon Aug 13 13:52:46 2012
@@ -199,9 +199,9 @@ public class TopGroupsResultTransformer 
             if (sortValue instanceof BytesRef) {
               UnicodeUtil.UTF8toUTF16((BytesRef)sortValue, spare);
               String indexedValue = spare.toString();
-              sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 0.0f));
+              sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 1.0f));
             } else if (sortValue instanceof String) {
-              sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 0.0f));
+              sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 1.0f));
             }
           }
           convertedSortValues[j] = sortValue;
@@ -252,9 +252,9 @@ public class TopGroupsResultTransformer 
           if (sortValue instanceof BytesRef) {
             UnicodeUtil.UTF8toUTF16((BytesRef)sortValue, spare);
             String indexedValue = spare.toString();
-            sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 0.0f));
+            sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable(indexedValue), 1.0f));
           } else if (sortValue instanceof String) {
-            sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 0.0f));
+            sortValue = fieldType.toObject(field.createField(fieldType.indexedToReadable((String) sortValue), 1.0f));
           }
         }
         convertedSortValues[j] = sortValue;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/endresulttransformer/GroupedEndResultTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/endresulttransformer/GroupedEndResultTransformer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/endresulttransformer/GroupedEndResultTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/search/grouping/endresulttransformer/GroupedEndResultTransformer.java Mon Aug 13 13:52:46 2012
@@ -69,7 +69,7 @@ public class GroupedEndResultTransformer
           SimpleOrderedMap<Object> groupResult = new SimpleOrderedMap<Object>();
           if (group.groupValue != null) {
             groupResult.add(
-                "groupValue", groupFieldType.toObject(groupField.createField(group.groupValue.utf8ToString(), 0.0f))
+                "groupValue", groupFieldType.toObject(groupField.createField(group.groupValue.utf8ToString(), 1.0f))
             );
           } else {
             groupResult.add("groupValue", null);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Mon Aug 13 13:52:46 2012
@@ -39,7 +39,7 @@ import javax.servlet.http.HttpServletReq
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -315,8 +315,8 @@ public class SolrDispatchFilter implemen
     String collection = corename;
     ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
     
-    CloudState cloudState = zkStateReader.getCloudState();
-    Map<String,Slice> slices = cloudState.getSlices(collection);
+    ClusterState clusterState = zkStateReader.getClusterState();
+    Map<String,Slice> slices = clusterState.getSlices(collection);
     if (slices == null) {
       return null;
     }
@@ -326,7 +326,7 @@ public class SolrDispatchFilter implemen
     done:
     for (Entry<String,Slice> entry : entries) {
       // first see if we have the leader
-      ZkNodeProps leaderProps = cloudState.getLeader(collection, entry.getKey());
+      ZkNodeProps leaderProps = clusterState.getLeader(collection, entry.getKey());
       if (leaderProps != null) {
         core = checkProps(cores, path, leaderProps);
       }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/ConjunctionSolrSpellChecker.java Mon Aug 13 13:52:46 2012
@@ -106,7 +106,7 @@ public class ConjunctionSolrSpellChecker
   }
   
   @Override
-  public void build(SolrCore core, SolrIndexSearcher searcher) {
+  public void build(SolrCore core, SolrIndexSearcher searcher) throws IOException {
     for (SolrSpellChecker c : checkers) {
       c.build(core, searcher);
     }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/DirectSolrSpellChecker.java Mon Aug 13 13:52:46 2012
@@ -168,11 +168,10 @@ public class DirectSolrSpellChecker exte
   }
   
   @Override
-  public void reload(SolrCore core, SolrIndexSearcher searcher)
-      throws IOException {}
+  public void reload(SolrCore core, SolrIndexSearcher searcher) throws IOException {}
 
   @Override
-  public void build(SolrCore core, SolrIndexSearcher searcher) {}
+  public void build(SolrCore core, SolrIndexSearcher searcher) throws IOException {}
 
   @Override
   public SpellingResult getSuggestions(SpellingOptions options)

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java Mon Aug 13 13:52:46 2012
@@ -59,17 +59,13 @@ public class FileBasedSpellChecker exten
   }
 
   @Override
-  public void build(SolrCore core, SolrIndexSearcher searcher) {
-    try {
-      loadExternalFileDictionary(core);
-      spellChecker.clearIndex();
-      // TODO: you should be able to specify the IWC params?
-      // TODO: if we enable this, codec gets angry since field won't exist in the schema
-      // config.setCodec(core.getCodec());
-      spellChecker.indexDictionary(dictionary, new IndexWriterConfig(core.getSolrConfig().luceneMatchVersion, null), false);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
+  public void build(SolrCore core, SolrIndexSearcher searcher) throws IOException {
+    loadExternalFileDictionary(core);
+    spellChecker.clearIndex();
+    // TODO: you should be able to specify the IWC params?
+    // TODO: if we enable this, codec gets angry since field won't exist in the schema
+    // config.setCodec(core.getCodec());
+    spellChecker.indexDictionary(dictionary, new IndexWriterConfig(core.getSolrConfig().luceneMatchVersion, null), false);
   }
 
   /**

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java Mon Aug 13 13:52:46 2012
@@ -73,31 +73,27 @@ public class IndexBasedSpellChecker exte
   }
 
   @Override
-  public void build(SolrCore core, SolrIndexSearcher searcher) {
+  public void build(SolrCore core, SolrIndexSearcher searcher) throws IOException {
     IndexReader reader = null;
-    try {
-      if (sourceLocation == null) {
-        // Load from Solr's index
-        reader = searcher.getIndexReader();
-      } else {
-        // Load from Lucene index at given sourceLocation
-        reader = this.reader;
-      }
-
-      // Create the dictionary
-      dictionary = new HighFrequencyDictionary(reader, field,
-          threshold);
-      // TODO: maybe whether or not to clear the index should be configurable?
-      // an incremental update is faster (just adds new terms), but if you 'expunged'
-      // old terms I think they might hang around.
-      spellChecker.clearIndex();
-      // TODO: you should be able to specify the IWC params?
-      // TODO: if we enable this, codec gets angry since field won't exist in the schema
-      // config.setCodec(core.getCodec());
-      spellChecker.indexDictionary(dictionary, new IndexWriterConfig(core.getSolrConfig().luceneMatchVersion, null), false);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
+    if (sourceLocation == null) {
+      // Load from Solr's index
+      reader = searcher.getIndexReader();
+    } else {
+      // Load from Lucene index at given sourceLocation
+      reader = this.reader;
     }
+
+    // Create the dictionary
+    dictionary = new HighFrequencyDictionary(reader, field,
+        threshold);
+    // TODO: maybe whether or not to clear the index should be configurable?
+    // an incremental update is faster (just adds new terms), but if you 'expunged'
+    // old terms I think they might hang around.
+    spellChecker.clearIndex();
+    // TODO: you should be able to specify the IWC params?
+    // TODO: if we enable this, codec gets angry since field won't exist in the schema
+    // config.setCodec(core.getCodec());
+    spellChecker.indexDictionary(dictionary, new IndexWriterConfig(core.getSolrConfig().luceneMatchVersion, null), false);
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/SolrSpellChecker.java Mon Aug 13 13:52:46 2012
@@ -167,7 +167,7 @@ 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);
+  public abstract void build(SolrCore core, SolrIndexSearcher searcher) throws IOException;
   
   /**
    * Get the value of {@link SpellingParams#SPELLCHECK_ACCURACY} if supported.  

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/suggest/Suggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/suggest/Suggester.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/suggest/Suggester.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/spelling/suggest/Suggester.java Mon Aug 13 13:52:46 2012
@@ -35,6 +35,7 @@ import org.apache.lucene.search.suggest.
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IOUtils;
 
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
@@ -121,7 +122,7 @@ public class Suggester extends SolrSpell
   }
   
   @Override
-  public void build(SolrCore core, SolrIndexSearcher searcher) {
+  public void build(SolrCore core, SolrIndexSearcher searcher) throws IOException {
     LOG.info("build()");
     if (sourceLocation == null) {
       reader = searcher.getIndexReader();
@@ -129,30 +130,26 @@ public class Suggester extends SolrSpell
     } else {
       try {
         dictionary = new FileDictionary(new InputStreamReader(
-                core.getResourceLoader().openResource(sourceLocation), "UTF-8"));
+                core.getResourceLoader().openResource(sourceLocation), IOUtils.CHARSET_UTF_8));
       } catch (UnsupportedEncodingException e) {
         // should not happen
         LOG.error("should not happen", e);
       }
     }
-    try {
-      lookup.build(dictionary);
-      if (storeDir != null) {
-        File target = new File(storeDir, factory.storeFileName());
-        if(!lookup.store(new FileOutputStream(target))) {
-          if (sourceLocation == null) {
-            assert reader != null && field != null;
-            LOG.error("Store Lookup build from index on field: " + field + " failed reader has: " + reader.maxDoc() + " docs");
-          } else {
-            LOG.error("Store Lookup build from sourceloaction: " + sourceLocation + " failed");
-          }
+
+    lookup.build(dictionary);
+    if (storeDir != null) {
+      File target = new File(storeDir, factory.storeFileName());
+      if(!lookup.store(new FileOutputStream(target))) {
+        if (sourceLocation == null) {
+          assert reader != null && field != null;
+          LOG.error("Store Lookup build from index on field: " + field + " failed reader has: " + reader.maxDoc() + " docs");
         } else {
-          LOG.info("Stored suggest data to: " + target.getAbsolutePath());
+          LOG.error("Store Lookup build from sourceloaction: " + sourceLocation + " failed");
         }
+      } else {
+        LOG.info("Stored suggest data to: " + target.getAbsolutePath());
       }
-
-    } catch (Exception e) {
-      LOG.error("Error while building or storing Suggester data", e);
     }
   }
 
@@ -161,8 +158,13 @@ public class Suggester extends SolrSpell
     LOG.info("reload()");
     if (dictionary == null && storeDir != null) {
       // this may be a firstSearcher event, try loading it
-      if (lookup.load(new FileInputStream(new File(storeDir, factory.storeFileName())))) {
-        return;  // loaded ok
+      FileInputStream is = new FileInputStream(new File(storeDir, factory.storeFileName()));
+      try {
+        if (lookup.load(is)) {
+          return;  // loaded ok
+        }
+      } finally {
+        IOUtils.closeWhileHandlingException(is);
       }
       LOG.debug("load failed, need to build Lookup again");
     }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/CommitTracker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/CommitTracker.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/CommitTracker.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/CommitTracker.java Mon Aug 13 13:52:46 2012
@@ -29,6 +29,7 @@ import org.apache.solr.common.params.Mod
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,8 +40,10 @@ import org.slf4j.LoggerFactory;
  * definitely change in the future, so the interface should not be relied-upon
  * 
  * Note: all access must be synchronized.
+ * 
+ * Public for tests.
  */
-final class CommitTracker implements Runnable {
+public final class CommitTracker implements Runnable {
   protected final static Logger log = LoggerFactory.getLogger(CommitTracker.class);
   
   // scheduler delay for maxDoc-triggered autocommits
@@ -50,8 +53,8 @@ final class CommitTracker implements Run
   private int docsUpperBound;
   private long timeUpperBound;
   
-  private final ScheduledExecutorService scheduler = Executors
-      .newScheduledThreadPool(1);
+  private final ScheduledExecutorService scheduler = 
+      Executors.newScheduledThreadPool(1, new DefaultSolrThreadFactory("commitScheduler"));
   private ScheduledFuture pending;
   
   // state
@@ -248,7 +251,8 @@ final class CommitTracker implements Run
     this.docsUpperBound = docsUpperBound;
   }
 
-  void setTimeUpperBound(long timeUpperBound) {
+  // only for testing - not thread safe
+  public void setTimeUpperBound(long timeUpperBound) {
     this.timeUpperBound = timeUpperBound;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java Mon Aug 13 13:52:46 2012
@@ -35,6 +35,10 @@ public final class DefaultSolrCoreState 
   private final boolean SKIP_AUTO_RECOVERY = Boolean.getBoolean("solrcloud.skip.autorecovery");
   
   private final Object recoveryLock = new Object();
+  
+  // protects pauseWriter and writerFree
+  private final Object writerPauseLock = new Object();
+  
   private int refCnt = 1;
   private SolrIndexWriter indexWriter = null;
   private DirectoryFactory directoryFactory;
@@ -55,62 +59,75 @@ public final class DefaultSolrCoreState 
   @Override
   public synchronized RefCounted<IndexWriter> getIndexWriter(SolrCore core)
       throws IOException {
-    while (pauseWriter) {
-      try {
-        wait();
-      } catch (InterruptedException e) {}
-    }
-    
-    if (indexWriter == null) {
-      indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2", false,
-          false);
-    }
-    if (refCntWriter == null) {
-      refCntWriter = new RefCounted<IndexWriter>(indexWriter) {
-        @Override
-        public void close() {
-          synchronized (DefaultSolrCoreState.this) {
-            writerFree = true;
-            DefaultSolrCoreState.this.notifyAll();
+    synchronized (writerPauseLock) {
+      if (core == null) {
+        // core == null is a signal to just return the current writer, or null
+        // if none.
+        if (refCntWriter != null) refCntWriter.incref();
+        return refCntWriter;
+      }
+      
+      while (pauseWriter) {
+        try {
+          writerPauseLock.wait();
+        } catch (InterruptedException e) {}
+      }
+      
+      if (indexWriter == null) {
+        indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2",
+            false, false);
+      }
+      if (refCntWriter == null) {
+        refCntWriter = new RefCounted<IndexWriter>(indexWriter) {
+          @Override
+          public void close() {
+            synchronized (writerPauseLock) {
+              writerFree = true;
+              writerPauseLock.notifyAll();
+            }
           }
-        }
-      };
+        };
+      }
+      writerFree = false;
+      writerPauseLock.notifyAll();
+      refCntWriter.incref();
+      return refCntWriter;
     }
-    writerFree = false;
-    notifyAll();
-    refCntWriter.incref();
-    return refCntWriter;
   }
 
   @Override
-  public synchronized void newIndexWriter(SolrCore core) throws IOException {
-    // we need to wait for the Writer to fall out of use
-    // first lets stop it from being lent out
-    pauseWriter = true;
-    // then lets wait until its out of use
-    while(!writerFree) {
-      try {
-        wait();
-      } catch (InterruptedException e) {}
-    }
+  public synchronized void newIndexWriter(SolrCore core, boolean rollback) throws IOException {
     
-    if (indexWriter != null) {
+    synchronized (writerPauseLock) {
+      // we need to wait for the Writer to fall out of use
+      // first lets stop it from being lent out
+      pauseWriter = true;
+      // then lets wait until its out of use
+      while (!writerFree) {
+        try {
+          writerPauseLock.wait();
+        } catch (InterruptedException e) {}
+      }
+      
       try {
-        indexWriter.close();
-      } catch (Exception e) {
-        SolrException.log(log, "Error closing old IndexWriter", e);
+        if (indexWriter != null) {
+          try {
+            indexWriter.close();
+          } catch (Throwable t) {
+            SolrException.log(log, "Error closing old IndexWriter", t);
+          }
+        }
+        
+        indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2",
+            false, true);
+        // we need to null this so it picks up the new writer next get call
+        refCntWriter = null;
+      } finally {
+        
+        pauseWriter = false;
+        writerPauseLock.notifyAll();
       }
     }
-    
-    try {
-      indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2", false,
-          true);
-      // we need to null this so it picks up the new writer next get call
-      refCntWriter = null;
-    } finally {
-      pauseWriter = false;
-      notifyAll();
-    }
   }
 
   @Override
@@ -154,7 +171,7 @@ public final class DefaultSolrCoreState 
   @Override
   public synchronized void rollbackIndexWriter(SolrCore core) throws IOException {
     indexWriter.rollback();
-    newIndexWriter(core);
+    newIndexWriter(core, true);
   }
   
   protected SolrIndexWriter createMainIndexWriter(SolrCore core, String name,
@@ -181,8 +198,9 @@ public final class DefaultSolrCoreState 
       return;
     }
     
-    cancelRecovery();
     synchronized (recoveryLock) {
+      cancelRecovery();
+      
       while (recoveryRunning) {
         try {
           recoveryLock.wait(1000);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Mon Aug 13 13:52:46 2012
@@ -343,7 +343,7 @@ public class DirectUpdateHandler2 extend
 
       // currently for testing purposes.  Do a delete of complete index w/o worrying about versions, don't log, clean up most state in update log, etc
       if (delAll && cmd.getVersion() == -Long.MAX_VALUE) {
-        synchronized (this) {
+        synchronized (solrCoreState.getUpdateLock()) {
           deleteAll();
           ulog.deleteAll();
           return;
@@ -356,7 +356,7 @@ public class DirectUpdateHandler2 extend
       // a realtime view of the index.  When a new searcher is opened after a DBQ, that
       // flag can be cleared.  If those thing happen concurrently, it's not thread safe.
       //
-      synchronized (this) {
+      synchronized (solrCoreState.getUpdateLock()) {
         if (delAll) {
           deleteAll();
         } else {
@@ -392,7 +392,7 @@ public class DirectUpdateHandler2 extend
     Term idTerm = new Term(idField.getName(), cmd.getIndexedId());
     
     // see comment in deleteByQuery
-    synchronized (this) {
+    synchronized (solrCoreState.getUpdateLock()) {
       RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
       try {
         IndexWriter writer = iw.get();
@@ -518,7 +518,7 @@ public class DirectUpdateHandler2 extend
         }
         
         if (!cmd.softCommit) {
-          synchronized (this) { // sync is currently needed to prevent preCommit
+          synchronized (solrCoreState.getUpdateLock()) { // sync is currently needed to prevent preCommit
                                 // from being called between preSoft and
                                 // postSoft... see postSoft comments.
             if (ulog != null) ulog.preCommit(cmd);
@@ -547,14 +547,14 @@ public class DirectUpdateHandler2 extend
 
       if (cmd.softCommit) {
         // ulog.preSoftCommit();
-        synchronized (this) {
+        synchronized (solrCoreState.getUpdateLock()) {
           if (ulog != null) ulog.preSoftCommit(cmd);
           core.getSearcher(true, false, waitSearcher, true);
           if (ulog != null) ulog.postSoftCommit(cmd);
         }
         // ulog.postSoftCommit();
       } else {
-        synchronized (this) {
+        synchronized (solrCoreState.getUpdateLock()) {
           if (ulog != null) ulog.preSoftCommit(cmd);
           if (cmd.openSearcher) {
             core.getSearcher(true, false, waitSearcher);
@@ -606,8 +606,8 @@ public class DirectUpdateHandler2 extend
   }
 
   @Override
-  public void newIndexWriter() throws IOException {
-    solrCoreState.newIndexWriter(core);
+  public void newIndexWriter(boolean rollback) throws IOException {
+    solrCoreState.newIndexWriter(core, rollback);
   }
   
   /**
@@ -705,7 +705,7 @@ public class DirectUpdateHandler2 extend
           // TODO: keep other commit callbacks from being called?
          //  this.commit(cmd);        // too many test failures using this method... is it because of callbacks?
 
-          synchronized (this) {
+          synchronized (solrCoreState.getUpdateLock()) {
             ulog.preCommit(cmd);
           }
 
@@ -714,7 +714,7 @@ public class DirectUpdateHandler2 extend
           commitData.put(SolrIndexWriter.COMMIT_TIME_MSEC_KEY, String.valueOf(System.currentTimeMillis()));
           writer.commit(commitData);
 
-          synchronized (this) {
+          synchronized (solrCoreState.getUpdateLock()) {
             ulog.postCommit(cmd);
           }
         }
@@ -823,4 +823,14 @@ public class DirectUpdateHandler2 extend
   public void incref() {
     solrCoreState.incref();
   }
+
+  // allow access for tests
+  public CommitTracker getCommitTracker() {
+    return commitTracker;
+  }
+
+  // allow access for tests
+  public CommitTracker getSoftCommitTracker() {
+    return softCommitTracker;
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java Mon Aug 13 13:52:46 2012
@@ -235,7 +235,7 @@ public class DocumentBuilder {
       SchemaField sfield = schema.getFieldOrNull(name);
       boolean used = false;
       float boost = field.getBoost();
-      boolean omitNorms = sfield != null && sfield.omitNorms();
+      boolean applyBoost = sfield != null && sfield.indexed() && !sfield.omitNorms();
       
       // Make sure it has the correct number
       if( sfield!=null && !sfield.multiValued() && field.getValueCount() > 1 ) {
@@ -244,9 +244,9 @@ public class DocumentBuilder {
               sfield.getName() + ": " +field.getValue() );
       }
       
-      if (omitNorms && boost != 1.0F) {
+      if (applyBoost == false && boost != 1.0F) {
         throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
-            "ERROR: "+getID(doc, schema)+"cannot set an index-time boost, norms are omitted for field " + 
+            "ERROR: "+getID(doc, schema)+"cannot set an index-time boost, unindexed or norms are omitted for field " + 
               sfield.getName() + ": " +field.getValue() );
       }
 
@@ -260,7 +260,7 @@ public class DocumentBuilder {
           hasField = true;
           if (sfield != null) {
             used = true;
-            addField(out, sfield, v, omitNorms ? 1F : docBoost*boost);
+            addField(out, sfield, v, applyBoost ? docBoost*boost : 1f);
           }
   
           // Check if we should copy this field to any other fields.
@@ -282,7 +282,7 @@ public class DocumentBuilder {
             if( val instanceof String && cf.getMaxChars() > 0 ) {
               val = cf.getLimitedValue((String)val);
             }
-            addField(out, destinationField, val, destinationField.omitNorms() ? 1F : docBoost*boost);
+            addField(out, destinationField, val, destinationField.indexed() && !destinationField.omitNorms() ? docBoost*boost : 1F);
           }
           
           // In lucene, the boost for a given field is the product of the 

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/PeerSync.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/PeerSync.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/PeerSync.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/PeerSync.java Mon Aug 13 13:52:46 2012
@@ -165,7 +165,7 @@ public class PeerSync  {
     String myURL = "";
 
     if (zkController != null) {
-      myURL = zkController.getZkServerAddress();
+      myURL = zkController.getBaseUrl();
     }
 
     // TODO: core name turns up blank in many tests - find URL if cloud enabled?

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java Mon Aug 13 13:52:46 2012
@@ -44,29 +44,28 @@ import org.apache.solr.common.cloud.ZkCo
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.AdjustableSemaphore;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-
-
 public class SolrCmdDistributor {
   private static final int MAX_RETRIES_ON_FORWARD = 6;
   public static Logger log = LoggerFactory.getLogger(SolrCmdDistributor.class);
   
   // TODO: shut this thing down
-  // TODO: this cannot be per instance...
-  static ThreadPoolExecutor commExecutor = new ThreadPoolExecutor(0,
-      Integer.MAX_VALUE, 5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
-      new DefaultSolrThreadFactory("cmdDistribExecutor"));
+  static ThreadPoolExecutor commExecutor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 5,
+      TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+      new DefaultSolrThreadFactory("cmdDistribExecutor"));;
 
   static final HttpClient client;
+  static AdjustableSemaphore semaphore = new AdjustableSemaphore(8);
   
   static {
     ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS, 200);
-    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, 8);
+    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS, 500);
+    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, 16);
     client = HttpClientUtil.createClient(params);
   }
   
@@ -91,8 +90,16 @@ public class SolrCmdDistributor {
     ModifiableSolrParams params;
   }
   
-  public SolrCmdDistributor() {
-   
+  public SolrCmdDistributor(int numHosts) {
+    int maxPermits = Math.max(8, (numHosts - 1) * 8);
+    
+    // limits how many tasks can actually execute at once
+    if (maxPermits != semaphore.getMaxPermits()) {
+      semaphore.setMaxPermits(maxPermits);
+    }
+
+    completionService = new ExecutorCompletionService<Request>(commExecutor);
+    pending = new HashSet<Future<Request>>();
   }
   
   public void finish() {
@@ -207,7 +214,7 @@ public class SolrCmdDistributor {
  
     for (Node node : nodes) {
       List<AddRequest> alist = adds.get(node);
-      if (alist == null || alist.size() < limit) return false;
+      if (alist == null || alist.size() < limit) continue;
   
       UpdateRequestExt ureq = new UpdateRequestExt();
       
@@ -242,7 +249,7 @@ public class SolrCmdDistributor {
     Set<Node> nodes = deletes.keySet();
     for (Node node : nodes) {
       List<DeleteRequest> dlist = deletes.get(node);
-      if (dlist == null || dlist.size() < limit) return false;
+      if (dlist == null || dlist.size() < limit) continue;
       UpdateRequestExt ureq = new UpdateRequestExt();
       
       ModifiableSolrParams combinedParams = new ModifiableSolrParams();
@@ -297,10 +304,7 @@ public class SolrCmdDistributor {
   }
   
   public void submit(final Request sreq) {
-    if (completionService == null) {
-      completionService = new ExecutorCompletionService<Request>(commExecutor);
-      pending = new HashSet<Future<Request>>();
-    }
+
     final String url = sreq.node.getUrl();
 
     Callable<Request> task = new Callable<Request>() {
@@ -332,11 +336,17 @@ public class SolrCmdDistributor {
           } else {
             clonedRequest.rspCode = -1;
           }
+        } finally {
+          semaphore.release();
         }
         return clonedRequest;
       }
     };
-    
+    try {
+      semaphore.acquire();
+    } catch (InterruptedException e) {
+      throw new RuntimeException();
+    }
     pending.add(completionService.submit(task));
     
   }
@@ -502,4 +512,7 @@ public class SolrCmdDistributor {
       return nodeProps;
     }
   }
+  
 }
+
+

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCoreState.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCoreState.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/SolrCoreState.java Mon Aug 13 13:52:46 2012
@@ -31,15 +31,21 @@ import org.apache.solr.util.RefCounted;
  * 
  */
 public abstract class SolrCoreState {
+  private final Object deleteLock = new Object();
+  
+  public Object getUpdateLock() {
+    return deleteLock;
+  }
   
   /**
    * Force the creation of a new IndexWriter using the settings from the given
    * SolrCore.
    * 
    * @param core
+   * @param rollback close IndexWriter if false, else rollback
    * @throws IOException
    */
-  public abstract void newIndexWriter(SolrCore core) throws IOException;
+  public abstract void newIndexWriter(SolrCore core, boolean rollback) throws IOException;
   
   /**
    * Get the current IndexWriter. If a new IndexWriter must be created, use the

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/TransactionLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/TransactionLog.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/TransactionLog.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/TransactionLog.java Mon Aug 13 13:52:46 2012
@@ -34,9 +34,11 @@ import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -149,7 +151,8 @@ public class TransactionLog {
       long start = raf.length();
       channel = raf.getChannel();
       os = Channels.newOutputStream(channel);
-      fos = FastOutputStream.wrap(os);
+      fos = new FastOutputStream(os, new byte[65536], 0);
+      // fos = FastOutputStream.wrap(os);
 
       if (openExisting) {
         if (start > 0) {
@@ -300,93 +303,119 @@ public class TransactionLog {
     numRecords++;
   }
 
+  private void checkWriteHeader(LogCodec codec, SolrInputDocument optional) throws IOException {
+
+    // Unsynchronized access.  We can get away with an unsynchronized access here
+    // since we will never get a false non-zero when the position is in fact 0.
+    // rollback() is the only function that can reset to zero, and it blocks updates.
+    if (fos.size() != 0) return;
+
+    synchronized (this) {
+      if (fos.size() != 0) return;  // check again while synchronized
+      if (optional != null) {
+        addGlobalStrings(optional.getFieldNames());
+      }
+      writeLogHeader(codec);
+    }
+  }
+
+  int lastAddSize;
 
   public long write(AddUpdateCommand cmd, int flags) {
     LogCodec codec = new LogCodec();
-    long pos = 0;
-    synchronized (this) {
-      try {
-        pos = fos.size();   // if we had flushed, this should be equal to channel.position()
-        SolrInputDocument sdoc = cmd.getSolrInputDocument();
+    SolrInputDocument sdoc = cmd.getSolrInputDocument();
 
-        if (pos == 0) { // TODO: needs to be changed if we start writing a header first
-          addGlobalStrings(sdoc.getFieldNames());
-          writeLogHeader(codec);
-          pos = fos.size();
-        }
+    try {
+      checkWriteHeader(codec, sdoc);
+
+      // adaptive buffer sizing
+      int bufSize = lastAddSize;    // unsynchronized access of lastAddSize should be fine
+      bufSize = Math.min(1024*1024, bufSize+(bufSize>>3)+256);
+
+      MemOutputStream out = new MemOutputStream(new byte[bufSize]);
+      codec.init(out);
+      codec.writeTag(JavaBinCodec.ARR, 3);
+      codec.writeInt(UpdateLog.ADD | flags);  // should just take one byte
+      codec.writeLong(cmd.getVersion());
+      codec.writeSolrInputDocument(cmd.getSolrInputDocument());
+      lastAddSize = (int)out.size();
+
+      synchronized (this) {
+        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
+        assert pos != 0;
 
         /***
-        System.out.println("###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
+         System.out.println("###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
          if (pos != fos.size()) {
-          throw new RuntimeException("ERROR" + "###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
-        }
+         throw new RuntimeException("ERROR" + "###writing at " + pos + " fos.size()=" + fos.size() + " raf.length()=" + raf.length());
+         }
          ***/
 
-        codec.init(fos);
-        codec.writeTag(JavaBinCodec.ARR, 3);
-        codec.writeInt(UpdateLog.ADD | flags);  // should just take one byte
-        codec.writeLong(cmd.getVersion());
-        codec.writeSolrInputDocument(cmd.getSolrInputDocument());
-
+        out.writeAll(fos);
         endRecord(pos);
         // fos.flushBuffer();  // flush later
         return pos;
-      } catch (IOException e) {
-        // TODO: reset our file pointer back to "pos", the start of this record.
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error logging add", e);
       }
+
+    } catch (IOException e) {
+      // TODO: reset our file pointer back to "pos", the start of this record.
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error logging add", e);
     }
   }
 
   public long writeDelete(DeleteUpdateCommand cmd, int flags) {
     LogCodec codec = new LogCodec();
-    synchronized (this) {
-      try {
-        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
-        if (pos == 0) {
-          writeLogHeader(codec);
-          pos = fos.size();
-        }
-        codec.init(fos);
-        codec.writeTag(JavaBinCodec.ARR, 3);
-        codec.writeInt(UpdateLog.DELETE | flags);  // should just take one byte
-        codec.writeLong(cmd.getVersion());
-        BytesRef br = cmd.getIndexedId();
-        codec.writeByteArray(br.bytes, br.offset, br.length);
 
+    try {
+      checkWriteHeader(codec, null);
+
+      BytesRef br = cmd.getIndexedId();
+
+      MemOutputStream out = new MemOutputStream(new byte[20 + br.length]);
+      codec.init(out);
+      codec.writeTag(JavaBinCodec.ARR, 3);
+      codec.writeInt(UpdateLog.DELETE | flags);  // should just take one byte
+      codec.writeLong(cmd.getVersion());
+      codec.writeByteArray(br.bytes, br.offset, br.length);
+
+      synchronized (this) {
+        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
+        assert pos != 0;
+        out.writeAll(fos);
         endRecord(pos);
         // fos.flushBuffer();  // flush later
-
         return pos;
-      } catch (IOException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       }
+
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
     }
+
   }
 
   public long writeDeleteByQuery(DeleteUpdateCommand cmd, int flags) {
     LogCodec codec = new LogCodec();
-    synchronized (this) {
-      try {
-        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
-        if (pos == 0) {
-          writeLogHeader(codec);
-          pos = fos.size();
-        }
-        codec.init(fos);
-        codec.writeTag(JavaBinCodec.ARR, 3);
-        codec.writeInt(UpdateLog.DELETE_BY_QUERY | flags);  // should just take one byte
-        codec.writeLong(cmd.getVersion());
-        codec.writeStr(cmd.query);
+    try {
+      checkWriteHeader(codec, null);
 
+      MemOutputStream out = new MemOutputStream(new byte[20 + (cmd.query.length())]);
+      codec.init(out);
+      codec.writeTag(JavaBinCodec.ARR, 3);
+      codec.writeInt(UpdateLog.DELETE_BY_QUERY | flags);  // should just take one byte
+      codec.writeLong(cmd.getVersion());
+      codec.writeStr(cmd.query);
+
+      synchronized (this) {
+        long pos = fos.size();   // if we had flushed, this should be equal to channel.position()
+        out.writeAll(fos);
         endRecord(pos);
         // fos.flushBuffer();  // flush later
-
         return pos;
+      }
       } catch (IOException e) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       }
-    }
+
   }
 
 
@@ -745,3 +774,32 @@ class ChannelFastInputStream extends Fas
   }
 }
 
+
+class MemOutputStream extends FastOutputStream {
+  public List<byte[]> buffers = new LinkedList<byte[]>();
+  public MemOutputStream(byte[] tempBuffer) {
+    super(null, tempBuffer, 0);
+  }
+
+  @Override
+  public void flush(byte[] arr, int offset, int len) throws IOException {
+    if (arr == buf && offset==0 && len==buf.length) {
+      buffers.add(buf);  // steal the buffer
+      buf = new byte[8192];
+    } else if (len > 0) {
+      byte[] newBuf = new byte[len];
+      System.arraycopy(arr, offset, newBuf, 0, len);
+      buffers.add(newBuf);
+    }
+  }
+
+  public void writeAll(FastOutputStream fos) throws IOException {
+    for (byte[] buffer : buffers) {
+      fos.write(buffer);
+    }
+    if (pos > 0) {
+      fos.write(buf, 0, pos);
+    }
+  }
+}
+

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateCommand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateCommand.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateCommand.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateCommand.java Mon Aug 13 13:52:46 2012
@@ -44,7 +44,7 @@ public abstract class UpdateCommand impl
 
   @Override
   public String toString() {
-    return name() + "{flags="+flags+",version="+version;
+    return name() + "{flags="+flags+",_version_="+version;
   }
 
   public long getVersion() {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateHandler.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/java/org/apache/solr/update/UpdateHandler.java Mon Aug 13 13:52:46 2012
@@ -127,9 +127,11 @@ public abstract class UpdateHandler impl
    * Called when the Writer should be opened again - eg when replication replaces
    * all of the index files.
    * 
+   * @param rollback IndexWriter if true else close
+   * 
    * @throws IOException
    */
-  public abstract void newIndexWriter() throws IOException;
+  public abstract void newIndexWriter(boolean rollback) throws IOException;
 
   public abstract SolrCoreState getSolrCoreState();