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 2013/02/17 01:45:24 UTC

svn commit: r1446985 [2/2] - in /lucene/dev/branches/lucene4765: ./ lucene/ lucene/core/ lucene/core/src/test/org/apache/lucene/search/ lucene/queries/ lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ solr/ solr/core/ solr/core/s...

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/StrField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/StrField.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/StrField.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/StrField.java Sun Feb 17 00:45:22 2013
@@ -17,20 +17,43 @@
 
 package org.apache.solr.schema;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.index.GeneralField;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.StorableField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
 
-import java.io.IOException;
-/**
- *
- */
 public class StrField extends PrimitiveFieldType {
+
+  @Override
+  protected void init(IndexSchema schema, Map<String,String> args) {
+    super.init(schema, args);
+  }
+
+  @Override
+  public List<StorableField> createFields(SchemaField field, Object value,
+      float boost) {
+    if (field.hasDocValues()) {
+      List<StorableField> fields = new ArrayList<StorableField>();
+      fields.add(createField(field, value, boost));
+      final BytesRef bytes = new BytesRef(value.toString());
+      final Field docValuesField = new SortedDocValuesField(field.getName(), bytes);
+      fields.add(docValuesField);
+      return fields;
+    } else {
+      return Collections.singletonList(createField(field, value, boost));
+    }
+  }
+
   @Override
   public SortField getSortField(SchemaField field,boolean reverse) {
     return getStringSort(field,reverse);
@@ -51,6 +74,14 @@ public class StrField extends PrimitiveF
   public Object toObject(SchemaField sf, BytesRef term) {
     return term.utf8ToString();
   }
+
+  @Override
+  public void checkSchemaField(SchemaField field) {
+    // change me when multi-valued doc values are supported
+    if (field.hasDocValues() && !(field.isRequired() || field.getDefaultValue() != null)) {
+      throw new IllegalStateException("Field " + this + " has doc values enabled, but has no default value and is not required");
+    }
+  }
 }
 
 

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieDateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieDateField.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieDateField.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieDateField.java Sun Feb 17 00:45:22 2013
@@ -17,11 +17,13 @@
 
 package org.apache.solr.schema;
 
+import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.solr.search.QParser;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.response.TextResponseWriter;
-import org.apache.lucene.index.GeneralField;
-import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.document.FieldType.NumericType;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.Query;
@@ -29,6 +31,7 @@ import org.apache.lucene.search.NumericR
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 
+import java.util.List;
 import java.util.Map;
 import java.util.Date;
 import java.io.IOException;
@@ -73,6 +76,10 @@ public class TrieDateField extends DateF
     return wrappedField.getPrecisionStep();
   }
 
+  @Override
+  public NumericType getNumericType() {
+    return wrappedField.getNumericType();
+  }
 
   @Override
   public void write(TextResponseWriter writer, String name, StorableField f) throws IOException {
@@ -130,6 +137,11 @@ public class TrieDateField extends DateF
   }
 
   @Override
+  public List<StorableField> createFields(SchemaField field, Object value, float boost) {
+    return wrappedField.createFields(field, value, boost);
+  }
+
+  @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
     return wrappedField.getRangeQuery(parser, field, min, max, minInclusive, maxInclusive);
   }
@@ -141,4 +153,10 @@ public class TrieDateField extends DateF
               max == null ? null : max.getTime(),
               minInclusive, maxInclusive);
   }
+
+  @Override
+  public void checkSchemaField(SchemaField field) {
+    wrappedField.checkSchemaField(field);
+  }
+
 }

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieField.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieField.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/TrieField.java Sun Feb 17 00:45:22 2013
@@ -17,7 +17,10 @@
 package org.apache.solr.schema;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 
@@ -29,13 +32,17 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.FloatField;
 import org.apache.lucene.document.IntField;
 import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.index.StorableField;
 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.*;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.NumericUtils;
@@ -101,8 +108,7 @@ public class TrieField extends Primitive
                 "Invalid type specified in schema.xml for field: " + args.get("name"), e);
       }
     }
-  
-    
+
     CharFilterFactory[] filterFactories = new CharFilterFactory[0];
     TokenFilterFactory[] tokenFilterFactories = new TokenFilterFactory[0];
     analyzer = new TokenizerChain(filterFactories, new TrieTokenizerFactory(type, precisionStep), tokenFilterFactories);
@@ -237,6 +243,23 @@ public class TrieField extends Primitive
   }
 
   @Override
+  public NumericType getNumericType() {
+    switch (type) {
+      case INTEGER:
+        return NumericType.INT;
+      case LONG:
+      case DATE:
+        return NumericType.LONG;
+      case FLOAT:
+        return NumericType.FLOAT;
+      case DOUBLE:
+        return NumericType.DOUBLE;
+      default:
+        throw new AssertionError();
+    }
+  }
+
+  @Override
   public Query getRangeQuery(QParser parser, SchemaField field, String min, String max, boolean minInclusive, boolean maxInclusive) {
     int ps = precisionStep;
     Query query = null;
@@ -473,8 +496,9 @@ public class TrieField extends Primitive
   public StorableField createField(SchemaField field, Object value, float boost) {
     boolean indexed = field.indexed();
     boolean stored = field.stored();
+    boolean docValues = field.hasDocValues();
 
-    if (!indexed && !stored) {
+    if (!indexed && !stored && !docValues) {
       if (log.isTraceEnabled())
         log.trace("Ignoring unindexed/unstored field: " + field);
       return null;
@@ -549,6 +573,28 @@ public class TrieField extends Primitive
     return f;
   }
 
+  @Override
+  public List<StorableField> createFields(SchemaField sf, Object value, float boost) {
+    if (sf.hasDocValues()) {
+      List<StorableField> fields = new ArrayList<StorableField>();
+      final StorableField field = createField(sf, value, boost);
+      fields.add(field);
+      final long bits;
+      if (field.numericValue() instanceof Integer || field.numericValue() instanceof Long) {
+        bits = field.numericValue().longValue();
+      } else if (field.numericValue() instanceof Float) {
+        bits = Float.floatToIntBits(field.numericValue().floatValue());
+      } else {
+        assert field.numericValue() instanceof Double;
+        bits = Double.doubleToLongBits(field.numericValue().doubleValue());
+      }
+      fields.add(new NumericDocValuesField(sf.getName(), bits));
+      return fields;
+    } else {
+      return Collections.singletonList(createField(sf, value, boost));
+    }
+  }
+
   public enum TrieTypes {
     INTEGER,
     LONG,
@@ -586,6 +632,13 @@ public class TrieField extends Primitive
     }
     return null;
   }
+
+  @Override
+  public void checkSchemaField(final SchemaField field) {
+    if (field.hasDocValues() && !(field.isRequired() || field.getDefaultValue() != null)) {
+      throw new IllegalStateException("Field " + this + " has doc values enabled, but has no default value and is not required");
+    }
+  }
 }
 
 class TrieDateFieldSource extends LongFieldSource {
@@ -605,14 +658,20 @@ class TrieDateFieldSource extends LongFi
   }
 
   @Override
-  public Object longToObject(long val) {
+  public Date longToObject(long val) {
     return new Date(val);
   }
 
   @Override
+  public String longToString(long val) {
+    return TrieField.dateField.toExternal(longToObject(val));
+  }
+
+  @Override
   public long externalToLong(String extVal) {
     return TrieField.dateField.parseMath(null, extVal).getTime();
   }
+
 }
 
 

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/UUIDField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/UUIDField.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/UUIDField.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/schema/UUIDField.java Sun Feb 17 00:45:22 2013
@@ -22,8 +22,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.UUID;
 
-import org.apache.lucene.index.GeneralField;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.search.SortField;
 import org.apache.solr.common.SolrException;

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java Sun Feb 17 00:45:22 2013
@@ -23,10 +23,7 @@ import java.util.List;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.StoredField;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.StorableField;
-import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
@@ -58,33 +55,19 @@ public class DocumentBuilder {
     // we don't check for a null val ourselves because a solr.FieldType
     // might actually want to map it to something.  If createField()
     // returns null, then we don't store the field.
-    if (sfield.isPolyField()) {
-      StorableField[] fields = sfield.createFields(val, boost);
-      if (fields.length > 0) {
-        if (!sfield.multiValued()) {
-          String oldValue = map.put(sfield.getName(), val);
-          if (oldValue != null) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "ERROR: multiple values encountered for non multiValued field " + sfield.getName()
-                    + ": first='" + oldValue + "' second='" + val + "'");
-          }
-        }
-        // Add each field
-        for (StorableField field : fields) {
-          doc.add((Field) field);
+    List<StorableField> fields = sfield.createFields(val, boost);
+    if (!fields.isEmpty()) {
+      if (!sfield.multiValued()) {
+        String oldValue = map.put(sfield.getName(), val);
+        if (oldValue != null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "ERROR: multiple values encountered for non multiValued field " + sfield.getName()
+                  + ": first='" + oldValue + "' second='" + val + "'");
         }
       }
-    } else {
-      StorableField field = sfield.createField(val, boost);
-      if (field != null) {
-        if (!sfield.multiValued()) {
-          String oldValue = map.put(sfield.getName(), val);
-          if (oldValue != null) {
-            throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"ERROR: multiple values encountered for non multiValued field " + sfield.getName()
-                    + ": first='" + oldValue + "' second='" + val + "'");
-          }
-        }
+      // Add each field
+      for (StorableField field : fields) {
+        doc.add((Field) field);
       }
-      doc.add((Field) field);
     }
 
   }
@@ -192,14 +175,8 @@ public class DocumentBuilder {
 
 
   private static void addField(Document doc, SchemaField field, Object val, float boost) {
-    if (field.isPolyField()) {
-      StorableField[] farr = field.getType().createFields(field, val, boost);
-      for (StorableField f : farr) {
-        if (f != null) doc.add((Field) f); // null fields are not added
-      }
-    } else {
-      StorableField f = field.createField(val, boost);
-      if (f != null) doc.add((Field) f);  // null fields are not added
+    for (StorableField f : field.getType().createFields(field, val, boost)) {
+      if (f != null) doc.add((Field) f); // null fields are not added
     }
   }
   

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateHandler.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateHandler.java Sun Feb 17 00:45:22 2013
@@ -22,6 +22,8 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Vector;
 
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrEventListener;
@@ -87,13 +89,14 @@ public abstract class UpdateHandler impl
   private void clearLog(PluginInfo ulogPluginInfo) {
     if (ulogPluginInfo == null) return;
     File tlogDir = UpdateLog.getTlogDir(core, ulogPluginInfo);
+    log.info("Clearing tlog files, tlogDir=" + tlogDir);
     if (tlogDir.exists()) {
       String[] files = UpdateLog.getLogList(tlogDir);
       for (String file : files) {
-        File f = new File(file);
+        File f = new File(tlogDir, file);
         boolean s = f.delete();
         if (!s) {
-          log.error("Could not remove tlog file:" + f);
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Could not remove tlog file:" + f.getAbsolutePath());
         }
       }
     }

Modified: lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/java/org/apache/solr/update/UpdateLog.java Sun Feb 17 00:45:22 2013
@@ -1398,16 +1398,16 @@ public class UpdateLog implements Plugin
   
   public static File getTlogDir(SolrCore core, PluginInfo info) {
     String dataDir = (String) info.initArgs.get("dir");
-    if (dataDir == null) {
-      String ulogDir = core.getCoreDescriptor().getUlogDir();
-      if (ulogDir != null) {
-        dataDir = ulogDir;
-      }
-      
-      if (dataDir == null || dataDir.length() == 0) {
-        dataDir = core.getDataDir();
-      }
+    
+    String ulogDir = core.getCoreDescriptor().getUlogDir();
+    if (ulogDir != null) {
+      dataDir = ulogDir;
     }
+    
+    if (dataDir == null || dataDir.length() == 0) {
+      dataDir = core.getDataDir();
+    }
+    
     return new File(dataDir, TLOG_NAME);
   }
   

Modified: lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema.xml Sun Feb 17 00:45:22 2013
@@ -567,7 +567,7 @@
 
    <field name="textgap" type="textgap" indexed="true" stored="true"/>
 
-   <field name="timestamp" type="date" indexed="true" stored="true" default="NOW" multiValued="false"/>
+   <field name="timestamp" type="date" indexed="true" stored="true" docValues="true" default="NOW" multiValued="false"/>
    <field name="multiDefault" type="string" indexed="true" stored="true" default="muLti-Default" multiValued="true"/>
    <field name="intDefault" type="int" indexed="true" stored="true" default="42" multiValued="false"/>
 

Modified: lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml Sun Feb 17 00:45:22 2013
@@ -20,18 +20,29 @@
   <fieldType name="string_pulsing" class="solr.StrField" postingsFormat="Pulsing41"/>
   <fieldType name="string_simpletext" class="solr.StrField" postingsFormat="SimpleText"/>
   <fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene41"/>
-    <fieldType name="string" class="solr.StrField" />
-  
+
+  <fieldType name="string_disk" class="solr.StrField" docValuesFormat="Disk" />
+  <fieldType name="string_memory" class="solr.StrField" docValuesFormat="Lucene42" />
+
+  <fieldType name="string" class="solr.StrField" />
+
  </types>
  <fields>
    <field name="string_pulsing_f" type="string_pulsing" indexed="true" stored="true" />
    <field name="string_simpletext_f" type="string_simpletext" indexed="true" stored="true" />
    <field name="string_standard_f" type="string_standard" indexed="true" stored="true" />
-   <field name="string_f" type="string" indexed="true" stored="true" />
+
+   <field name="string_disk_f" type="string_disk" indexed="false" stored="false" docValues="true" default="" />
+   <field name="string_memory_f" type="string_memory" indexed="false" stored="false" docValues="true" default="" />
+
+   <field name="string_f" type="string" indexed="true" stored="true" docValues="true" required="true"/>
+
    <dynamicField name="*_simple" type="string_simpletext"  indexed="true" stored="true"/>
    <dynamicField name="*_pulsing" type="string_pulsing"  indexed="true" stored="true"/>
    <dynamicField name="*_standard" type="string_standard"  indexed="true" stored="true"/>
-   
+
+   <dynamicField name="*_disk" type="string_disk" indexed="false" stored="false" docValues="true" default="" />
+   <dynamicField name="*_memory" type="string_memory" indexed="false" stored="false" docValues="true" default="" />
  </fields>
   <defaultSearchField>string_f</defaultSearchField>
  <uniqueKey>string_f</uniqueKey>

Modified: lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml Sun Feb 17 00:45:22 2013
@@ -18,10 +18,21 @@
 -->
 
 <config>
+  <jmx />
+
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
-  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
+  
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}">
+    <!-- used to keep RAM reqs down for HdfsDirectoryFactory -->
+    <int name="solr.hdfs.blockcache.blocksperbank">${solr.hdfs.blockcache.blocksperbank:1024}</int>
+  </directoryFactory>
+  
   <dataDir>${solr.data.dir:}</dataDir>
 
+  <indexConfig>
+    <lockType>${solr.lock.type:native}</lockType>
+  </indexConfig>
+
   <!-- an update processor the explicitly excludes distrib to test
        clean errors when people attempt atomic updates w/o it
   -->
@@ -38,12 +49,14 @@
       <str name="omitHeader">true</str>
     </lst>
   </requestHandler>
+  
+  <requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" /> 
 
   <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
 
   <updateHandler class="solr.DirectUpdateHandler2">
     <updateLog>
-      <!-- <str name="dir">/tmp/solr/</str> -->
+      <str name="dir">${solr.ulog.dir:}</str>
     </updateLog>
   </updateHandler>
 
@@ -72,4 +85,35 @@
 
   <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
 
+  <updateRequestProcessorChain name="distrib-dup-test-chain-explicit">
+    <!-- explicit test using processors before and after distrib -->
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_A_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_B_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>  
+
+  <updateRequestProcessorChain name="distrib-dup-test-chain-implicit">
+    <!-- implicit test w/o distrib declared-->
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_A_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RegexReplaceProcessorFactory">
+      <str name="fieldName">regex_dup_B_s</str>
+      <str name="pattern">x</str>
+      <str name="replacement">x_x</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>  
+
 </config>

Modified: lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/solr.xml?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/solr.xml (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test-files/solr/solr.xml Sun Feb 17 00:45:22 2013
@@ -30,10 +30,10 @@
   -->
   <cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" 
          hostContext="${hostContext:solr}" zkClientTimeout="${solr.zkclienttimeout:30000}" numShards="${numShards:3}" shareSchema="${shareSchema:false}" 
-         distribUpdateConnTimeout="${distribUpdateConnTimeout:15000}" distribUpdateSoTimeout="${distribUpdateSoTimeout:60000}">
+         distribUpdateConnTimeout="${distribUpdateConnTimeout:15000}" distribUpdateSoTimeout="${distribUpdateSoTimeout:120000}">
     <core name="collection1" instanceDir="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" schema="${schema:schema.xml}"/>
     <shardHandlerFactory name="shardHandlerFactory" class="HttpShardHandlerFactory">
-      <int name="socketTimeout">${socketTimeout:60000}</int>
+      <int name="socketTimeout">${socketTimeout:120000}</int>
       <int name="connTimeout">${connTimeout:15000}</int>
     </shardHandlerFactory>
   </cores>

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java Sun Feb 17 00:45:22 2013
@@ -46,7 +46,7 @@ import org.slf4j.LoggerFactory;
 public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase {
   public static Logger log = LoggerFactory.getLogger(ChaosMonkeyNothingIsSafeTest.class);
   
-  private static final int BASE_RUN_LENGTH = 60000;
+  private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
 
   @BeforeClass
   public static void beforeSuperClass() {
@@ -138,8 +138,15 @@ public class ChaosMonkeyNothingIsSafeTes
       }
       
       chaosMonkey.startTheMonkey(true, 10000);
-      //int runLength = atLeast(BASE_RUN_LENGTH);
-      int runLength = BASE_RUN_LENGTH;
+
+      long runLength;
+      if (RUN_LENGTH != -1) {
+        runLength = RUN_LENGTH;
+      } else {
+        int[] runTimes = new int[] {5000,6000,10000,15000,15000,30000,30000,45000,90000,120000};
+        runLength = runTimes[random().nextInt(runTimes.length - 1)];
+      }
+      
       try {
         Thread.sleep(runLength);
       } finally {
@@ -172,7 +179,7 @@ public class ChaosMonkeyNothingIsSafeTes
       
       // make sure we again have leaders for each shard
       for (int j = 1; j < sliceCount; j++) {
-        zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+        zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 30000);
       }
       
       commit();

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java Sun Feb 17 00:45:22 2013
@@ -20,7 +20,6 @@ package org.apache.solr.cloud;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.lucene.util.LuceneTestCase.BadApple;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.common.SolrInputDocument;
@@ -37,8 +36,7 @@ import org.junit.BeforeClass;
 @Slow
 public class ChaosMonkeySafeLeaderTest extends AbstractFullDistribZkTestBase {
   
-  private static final int BASE_RUN_LENGTH = 120000;
-  private static final int RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", Integer.toString(BASE_RUN_LENGTH)));
+  private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
 
   @BeforeClass
   public static void beforeSuperClass() {
@@ -104,10 +102,18 @@ public class ChaosMonkeySafeLeaderTest e
     }
     
     chaosMonkey.startTheMonkey(false, 500);
-    int runLength = RUN_LENGTH;
-    Thread.sleep(runLength);
-    
-    chaosMonkey.stopTheMonkey();
+    long runLength;
+    if (RUN_LENGTH != -1) {
+      runLength = RUN_LENGTH;
+    } else {
+      int[] runTimes = new int[] {5000,6000,10000,15000,15000,30000,30000,45000,90000,120000};
+      runLength = runTimes[random().nextInt(runTimes.length - 1)];
+    }
+    try {
+      Thread.sleep(runLength);
+    } finally {
+      chaosMonkey.stopTheMonkey();
+    }
     
     for (StopableIndexingThread indexThread : threads) {
       indexThread.safeStop();

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/cloud/RecoveryZkTest.java Sun Feb 17 00:45:22 2013
@@ -50,30 +50,37 @@ public class RecoveryZkTest extends Abst
     
     // start a couple indexing threads
     
-    indexThread = new StopableIndexingThread(0, true);
+    int[] maxDocList = new int[] {300, 700, 1200, 1350, 5000, 15000};
+    
+    int maxDoc = maxDocList[random().nextInt(maxDocList.length - 1)];
+    
+    indexThread = new StopableIndexingThread(0, true, maxDoc);
     indexThread.start();
     
-    indexThread2 = new StopableIndexingThread(10000, true);
+    indexThread2 = new StopableIndexingThread(10000, true, maxDoc);
     
     indexThread2.start();
 
     // give some time to index...
-    Thread.sleep(atLeast(2000));   
-    
+    int[] waitTimes = new int[] {2000, 3000, 5000};
+    Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
+     
     // bring shard replica down
     JettySolrRunner replica = chaosMonkey.stopShard("shard1", 1).jetty;
 
     
     // wait a moment - lets allow some docs to be indexed so replication time is non 0
-    Thread.sleep(atLeast(2000));
+    Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
     
     // bring shard replica up
     replica.start();
     
     // make sure replication can start
-    Thread.sleep(1500);
+    Thread.sleep(3000);
     ZkStateReader zkStateReader = cloudClient.getZkStateReader();
-    waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, false, true);
+    
+    // give some time for replication to complete
+    Thread.sleep(5000);
     
     // stop indexing threads
     indexThread.safeStop();
@@ -86,12 +93,10 @@ public class RecoveryZkTest extends Abst
   
     waitForThingsToLevelOut(30);
     
-    Thread.sleep(1000);
+    Thread.sleep(2000);
     
     waitForThingsToLevelOut(30);
     
-    Thread.sleep(5000);
-    
     waitForRecoveriesToFinish(DEFAULT_COLLECTION, zkStateReader, false, true);
 
     // test that leader and replica have same doc count

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java Sun Feb 17 00:45:22 2013
@@ -20,6 +20,7 @@ package org.apache.solr.core;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.schema.SchemaField;
@@ -47,7 +48,21 @@ public class TestCodecSupport extends So
     assertEquals("Lucene41", format.getPostingsFormatForField(schemaField.getName()).getName());
   }
 
-  public void testDynamicFields() {
+  public void testDocValuesFormats() {
+    Codec codec = h.getCore().getCodec();
+    Map<String, SchemaField> fields = h.getCore().getSchema().getFields();
+    SchemaField schemaField = fields.get("string_disk_f");
+    PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
+    assertEquals("Disk", format.getDocValuesFormatForField(schemaField.getName()).getName());
+    schemaField = fields.get("string_memory_f");
+    assertEquals("Lucene42",
+        format.getDocValuesFormatForField(schemaField.getName()).getName());
+    schemaField = fields.get("string_f");
+    assertEquals("Lucene42",
+        format.getDocValuesFormatForField(schemaField.getName()).getName());
+  }
+
+  public void testDynamicFieldsPostingsFormats() {
     Codec codec = h.getCore().getCodec();
     PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();
 
@@ -59,6 +74,16 @@ public class TestCodecSupport extends So
     assertEquals("Lucene41", format.getPostingsFormatForField("bar_standard").getName());
   }
 
+  public void testDynamicFieldsDocValuesFormats() {
+    Codec codec = h.getCore().getCodec();
+    PerFieldDocValuesFormat format = (PerFieldDocValuesFormat) codec.docValuesFormat();
+
+    assertEquals("Disk", format.getDocValuesFormatForField("foo_disk").getName());
+    assertEquals("Disk", format.getDocValuesFormatForField("bar_disk").getName());
+    assertEquals("Lucene42", format.getDocValuesFormatForField("foo_memory").getName());
+    assertEquals("Lucene42", format.getDocValuesFormatForField("bar_memory").getName());
+  }
+
   public void testUnknownField() {
     Codec codec = h.getCore().getCodec();
     PerFieldPostingsFormat format = (PerFieldPostingsFormat) codec.postingsFormat();

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/handler/component/StatsComponentTest.java Sun Feb 17 00:45:22 2013
@@ -75,6 +75,7 @@ public class StatsComponentTest extends 
   public void doTestFieldStatisticsResult(String f) throws Exception {
     assertU(adoc("id", "1", f, "-10"));
     assertU(adoc("id", "2", f, "-20"));
+    assertU(commit());
     assertU(adoc("id", "3", f, "-30"));
     assertU(adoc("id", "4", f, "-40"));
     assertU(commit());
@@ -205,6 +206,7 @@ public class StatsComponentTest extends 
   public void doTestFieldStatisticsMissingResult(String f) throws Exception {
     assertU(adoc("id", "1", f, "-10"));
     assertU(adoc("id", "2", f, "-20"));
+    assertU(commit());
     assertU(adoc("id", "3"));
     assertU(adoc("id", "4", f, "-40"));
     assertU(commit());
@@ -224,6 +226,7 @@ public class StatsComponentTest extends 
   public void doTestFacetStatisticsResult(String f) throws Exception {
     assertU(adoc("id", "1", f, "10", "active_s", "true",  "other_s", "foo"));
     assertU(adoc("id", "2", f, "20", "active_s", "true",  "other_s", "bar"));
+    assertU(commit());
     assertU(adoc("id", "3", f, "30", "active_s", "false", "other_s", "foo"));
     assertU(adoc("id", "4", f, "40", "active_s", "false", "other_s", "foo"));
     assertU(commit());
@@ -257,6 +260,7 @@ public class StatsComponentTest extends 
   public void doTestFacetStatisticsMissingResult(String f) throws Exception {
       assertU(adoc("id", "1", f, "10", "active_s", "true"));
       assertU(adoc("id", "2", f, "20", "active_s", "true"));
+      assertU(commit());
       assertU(adoc("id", "3", "active_s", "false"));
       assertU(adoc("id", "4", f, "40", "active_s", "false"));
       assertU(commit());
@@ -288,6 +292,7 @@ public class StatsComponentTest extends 
     SolrCore core = h.getCore();
     assertU(adoc("id", "1"));
     assertU(adoc("id", "2"));
+    assertU(commit());
     assertU(adoc("id", "3"));
     assertU(adoc("id", "4"));
     assertU(commit());
@@ -307,6 +312,7 @@ public class StatsComponentTest extends 
     SolrCore core = h.getCore();
     assertU(adoc("id", "1"));
     assertU(adoc("id", "2"));
+    assertU(commit());
     assertU(adoc("id", "3"));
     assertU(adoc("id", "4"));
     assertU(commit());
@@ -328,6 +334,7 @@ public class StatsComponentTest extends 
 
     assertU(adoc("id", "1"));
     assertU(adoc("id", "2"));
+    assertU(commit());
     assertU(adoc("id", "3"));
     assertU(commit());
 
@@ -347,6 +354,7 @@ public class StatsComponentTest extends 
     SchemaField foo_ss = core.getSchema().getField("foo_ss");
 
     assertU(adoc("id", "1", "active_i", "1", "foo_ss", "aa" ));
+    assertU(commit());
     assertU(adoc("id", "2", "active_i", "1", "foo_ss", "bb" ));
     assertU(adoc("id", "3", "active_i", "5", "foo_ss", "aa" ));
     assertU(commit());

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java Sun Feb 17 00:45:22 2013
@@ -19,8 +19,6 @@ package org.apache.solr.schema;
 
 import org.apache.solr.core.AbstractBadConfigTestBase;
 
-import java.util.regex.Pattern;
-
 public class BadIndexSchemaTest extends AbstractBadConfigTestBase {
 
   private void doTest(final String schema, final String errString) 
@@ -83,5 +81,12 @@ public class BadIndexSchemaTest extends 
     doTest("bad-schema-codec-global-vs-ft-mismatch.xml", "codec does not support");
   }
 
+  public void testDocValuesNotRequiredNoDefault() throws Exception {
+    doTest("bad-schema-docValues-not-required-no-default.xml", "has no default value and is not required");
+  }
+
+  public void testDocValuesUnsupported() throws Exception {
+    doTest("bad-schema-unsupported-docValues.xml", "does not support doc values");
+  }
 
 }

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/CurrencyFieldTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/CurrencyFieldTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/CurrencyFieldTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/CurrencyFieldTest.java Sun Feb 17 00:45:22 2013
@@ -23,6 +23,7 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.List;
 import java.util.Random;
 import java.util.Set;
 
@@ -71,18 +72,18 @@ public class CurrencyFieldTest extends S
     FieldType tmp = amount.getType();
     assertTrue(tmp instanceof CurrencyField);
     String currencyValue = "1.50,EUR";
-    StorableField[] fields = amount.createFields(currencyValue, 2);
-    assertEquals(fields.length, 3);
+    List<StorableField> fields = amount.createFields(currencyValue, 2);
+    assertEquals(fields.size(), 3);
 
     // First field is currency code, second is value, third is stored.
     for (int i = 0; i < 3; i++) {
-      boolean hasValue = fields[i].readerValue() != null
-              || fields[i].numericValue() != null
-              || fields[i].stringValue() != null;
-      assertTrue("Doesn't have a value: " + fields[i], hasValue);
+      boolean hasValue = fields.get(i).readerValue() != null
+              || fields.get(i).numericValue() != null
+              || fields.get(i).stringValue() != null;
+      assertTrue("Doesn't have a value: " + fields.get(i), hasValue);
     }
 
-    assertEquals(schema.getFieldTypeByName("string").toExternal(fields[2]), "1.50,EUR");
+    assertEquals(schema.getFieldTypeByName("string").toExternal(fields.get(2)), "1.50,EUR");
     
     // A few tests on the provider directly
     ExchangeRateProvider p = ((CurrencyField) tmp).getProvider();

Modified: lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/schema/PolyFieldTest.java Sun Feb 17 00:45:22 2013
@@ -16,8 +16,9 @@ package org.apache.solr.schema;
  * limitations under the License.
  */
 
+import java.util.List;
+
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -84,14 +85,14 @@ public class PolyFieldTest extends SolrT
     assertEquals(pt.getDimension(), 2);
     double[] xy = new double[]{35.0, -79.34};
     String point = xy[0] + "," + xy[1];
-    StorableField[] fields = home.createFields(point, 2);
-    assertEquals(fields.length, 3);//should be 3, we have a stored field
+    List<StorableField> fields = home.createFields(point, 2);
+    assertEquals(fields.size(), 3);//should be 3, we have a stored field
     //first two fields contain the values, third is just stored and contains the original
     for (int i = 0; i < 3; i++) {
-      boolean hasValue = fields[i].binaryValue() != null
-          || fields[i].stringValue() != null
-          || fields[i].numericValue() != null;
-      assertTrue("Doesn't have a value: " + fields[i], hasValue);
+      boolean hasValue = fields.get(i).binaryValue() != null
+          || fields.get(i).stringValue() != null
+          || fields.get(i).numericValue() != null;
+      assertTrue("Doesn't have a value: " + fields.get(i), hasValue);
     }
     /*assertTrue("first field " + fields[0].tokenStreamValue() +  " is not 35.0", pt.getSubType().toExternal(fields[0]).equals(String.valueOf(xy[0])));
     assertTrue("second field is not -79.34", pt.getSubType().toExternal(fields[1]).equals(String.valueOf(xy[1])));
@@ -101,7 +102,7 @@ public class PolyFieldTest extends SolrT
     home = schema.getField("home_ns");
     assertNotNull(home);
     fields = home.createFields(point, 2);
-    assertEquals(fields.length, 2);//should be 2, since we aren't storing
+    assertEquals(fields.size(), 2);//should be 2, since we aren't storing
 
     home = schema.getField("home_ns");
     assertNotNull(home);

Modified: lucene/dev/branches/lucene4765/solr/example/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/example/solr/collection1/conf/schema.xml?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/example/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene4765/solr/example/solr/collection1/conf/schema.xml Sun Feb 17 00:45:22 2013
@@ -70,6 +70,15 @@
        <types> fieldType section
      indexed: true if this field should be indexed (searchable or sortable)
      stored: true if this field should be retrievable
+     docValues: true if this field should have doc values. Doc values are
+       useful for faceting, grouping, sorting and function queries. Although not
+       required, doc values will make the index faster to load, more
+       NRT-friendly and more memory-efficient. They however come with some
+       limitations: they are currently only supported by StrField, UUIDField
+       and all Trie*Fields, and depending on the field type, they might
+       require the field to be single-valued, be required or have a default
+       value (check the documentation of the field type you're interested in
+       for more information)
      multiValued: true if this field may contain multiple values per document
      omitNorms: (expert) set to true to omit the norms associated with
        this field (this disables length normalization and index-time
@@ -156,6 +165,17 @@
 
    <field name="_version_" type="long" indexed="true" stored="true"/>
 
+   <!--
+     Some fields such as popularity and manu_exact could be modified to
+     leverage doc values:
+     <field name="popularity" type="int" indexed="true" stored="true" docValues="true" default="0" />
+     <field name="manu_exact" type="string" indexed="false" stored="false" docValues="true" default="" />
+
+     Although it would make indexing slightly slower and the index bigger, it
+     would also make the index faster to load, more memory-efficient and more
+     NRT-friendly.
+     -->
+
    <!-- Uncommenting the following will create a "timestamp" field using
         a default value of "NOW" to indicate when each document was indexed.
      -->
@@ -282,7 +302,10 @@
        standard package such as org.apache.solr.analysis
     -->
 
-    <!-- The StrField type is not analyzed, but indexed/stored verbatim. -->
+    <!-- The StrField type is not analyzed, but indexed/stored verbatim.
+       It supports doc values but in that case the field needs to be
+       single-valued and either required or have a default value.
+      -->
     <fieldType name="string" class="solr.StrField" sortMissingLast="true" />
 
     <!-- boolean type: "true" or "false" -->
@@ -306,6 +329,9 @@
 
     <!--
       Default numeric field types. For faster range queries, consider the tint/tfloat/tlong/tdouble types.
+
+      These fields support doc values, but they require the field to be
+      single-valued and either be required or have a default value.
     -->
     <fieldType name="int" class="solr.TrieIntField" precisionStep="0" positionIncrementGap="0"/>
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" positionIncrementGap="0"/>

Modified: lucene/dev/branches/lucene4765/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java (original)
+++ lucene/dev/branches/lucene4765/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java Sun Feb 17 00:45:22 2013
@@ -52,6 +52,10 @@ public class CloudSolrServerTest extends
     
   }
   
+  protected String getCloudSolrConfig() {
+    return "solrconfig.xml";
+  }
+  
   @Override
   public String getSolrHome() {
     return SOLR_HOME;

Modified: lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java Sun Feb 17 00:45:22 2013
@@ -67,13 +67,17 @@ public abstract class AbstractDistribZkT
 
     String schema = getSchemaFile();
     if (schema == null) schema = "schema.xml";
-    AbstractZkTestCase.buildZooKeeper(zkServer.getZkHost(), zkServer.getZkAddress(), "solrconfig.xml", schema);
+    AbstractZkTestCase.buildZooKeeper(zkServer.getZkHost(), zkServer.getZkAddress(), getCloudSolrConfig(), schema);
 
     // set some system properties for use by tests
     System.setProperty("solr.test.sys.prop1", "propone");
     System.setProperty("solr.test.sys.prop2", "proptwo");
   }
   
+  protected String getCloudSolrConfig() {
+    return "solrconfig-tlog.xml";
+  }
+  
   @Override
   protected void createServers(int numShards) throws Exception {
     // give everyone there own solrhome
@@ -207,7 +211,6 @@ public abstract class AbstractDistribZkT
     if (DEBUG) {
       printLayout();
     }
-    zkServer.shutdown();
     System.clearProperty("zkHost");
     System.clearProperty("collection");
     System.clearProperty("enable.update.log");
@@ -217,6 +220,7 @@ public abstract class AbstractDistribZkT
     System.clearProperty("solr.test.sys.prop2");
     resetExceptionIgnores();
     super.tearDown();
+    zkServer.shutdown();
   }
   
   protected void printLayout() throws Exception {

Modified: lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1446985&r1=1446984&r2=1446985&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene4765/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Sun Feb 17 00:45:22 2013
@@ -1140,21 +1140,34 @@ public abstract class AbstractFullDistri
     protected final List<Integer> deletes = new ArrayList<Integer>();
     protected final AtomicInteger fails = new AtomicInteger();
     protected boolean doDeletes;
+    private int numCycles;
     
     public StopableIndexingThread(int startI, boolean doDeletes) {
+      this(startI, doDeletes, -1);
+    }
+    
+    public StopableIndexingThread(int startI, boolean doDeletes, int numCycles) {
       super("StopableIndexingThread");
       this.startI = startI;
       this.doDeletes = doDeletes;
+      this.numCycles = numCycles;
       setDaemon(true);
     }
     
     @Override
     public void run() {
       int i = startI;
+      int numDone = 0;
       int numDeletes = 0;
       int numAdds = 0;
       
       while (true && !stop) {
+        if (numCycles != -1) {
+          if (numDone > numCycles) {
+            break;
+          }
+        }
+        ++numDone;
         ++i;
         boolean addFailed = false;
         
@@ -1202,6 +1215,7 @@ public abstract class AbstractFullDistri
     
     @Override
     public void safeStop() {
+      System.out.println("safe stop:");
       stop = true;
     }