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/10/21 20:58:44 UTC

svn commit: r1534320 [38/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java Mon Oct 21 18:58:24 2013
@@ -28,9 +28,12 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.solr.common.cloud.DocCollection.DOC_ROUTER;
+
 /**
  * Class to partition int range into n ranges.
  * @lucene.experimental
@@ -38,12 +41,41 @@ import java.util.Map;
 public abstract class DocRouter {
   public static final String DEFAULT_NAME = CompositeIdRouter.NAME;
   public static final DocRouter DEFAULT = new CompositeIdRouter();
-  public static final String ROUTE_FIELD = "routeField";
 
-  public static DocRouter getDocRouter(Object routerSpec) {
-    DocRouter router = routerMap.get(routerSpec);
+
+  public static DocRouter getDocRouter(Object routerName) {
+    DocRouter router = routerMap.get(routerName);
     if (router != null) return router;
-    throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown document router '"+ routerSpec + "'");
+    throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown document router '"+ routerName + "'");
+  }
+
+  protected String getRouteField(DocCollection coll){
+    if(coll == null) return null;
+    Object o = coll.get(DOC_ROUTER);
+    if (o instanceof String) {
+      return null;
+      //old format. cannot have a routefield. Ignore it
+    }
+    Map m = (Map) o;
+    if(m == null) return null;
+    return (String) m.get("field");
+
+  }
+
+  public static Map<String,Object> getRouterSpec(ZkNodeProps props){
+    Map<String,Object> map =  new LinkedHashMap<String, Object>();
+    for (String s : props.keySet()) {
+      if(s.startsWith("router.")){
+        map.put(s.substring(7), props.get(s));
+      }
+    }
+    Object o = props.get("router");
+    if (o instanceof String) {
+      map.put("name", o);
+    } else if (map.get("name") == null) {
+      map.put("name", DEFAULT_NAME);
+    }
+    return  map;
   }
 
   // currently just an implementation detail...

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java Mon Oct 21 18:58:24 2013
@@ -30,19 +30,19 @@ public abstract class HashBasedRouter ex
   @Override
   public Slice getTargetSlice(String id, SolrInputDocument sdoc, SolrParams params, DocCollection collection) {
     if (id == null) id = getId(sdoc, params);
-    int hash = sliceHash(id, sdoc, params);
+    int hash = sliceHash(id, sdoc, params,collection);
     return hashToSlice(hash, collection);
   }
 
   @Override
   public boolean isTargetSlice(String id, SolrInputDocument sdoc, SolrParams params, String shardId, DocCollection collection) {
     if (id == null) id = getId(sdoc, params);
-    int hash = sliceHash(id, sdoc, params);
+    int hash = sliceHash(id, sdoc, params, collection);
     Range range = collection.getSlice(shardId).getRange();
     return range != null && range.includes(hash);
   }
 
-  public int sliceHash(String id, SolrInputDocument sdoc, SolrParams params) {
+  public int sliceHash(String id, SolrInputDocument sdoc, SolrParams params, DocCollection collection) {
     return Hash.murmurhash3_x86_32(id, 0, id.length(), 0);
   }
 

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ImplicitDocRouter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ImplicitDocRouter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ImplicitDocRouter.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ImplicitDocRouter.java Mon Oct 21 18:58:24 2013
@@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-
 import static org.apache.solr.common.params.ShardParams._ROUTE_;
 
 /** This document router is for custom sharding
@@ -42,7 +41,7 @@ public class ImplicitDocRouter extends D
   public Slice getTargetSlice(String id, SolrInputDocument sdoc, SolrParams params, DocCollection collection) {
     String shard = null;
     if (sdoc != null) {
-      String f = collection.getStr(ROUTE_FIELD);
+      String f = getRouteField(collection);
       if(f !=null) {
         Object o = sdoc.getFieldValue(f);
         if (o != null) shard = o.toString();

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java Mon Oct 21 18:58:24 2013
@@ -36,6 +36,8 @@ public class Slice extends ZkNodeProps {
   public static String ACTIVE = "active";
   public static String INACTIVE = "inactive";
   public static String CONSTRUCTION = "construction";
+  public static String RECOVERY = "recovery";
+  public static String PARENT = "parent";
 
   private final String name;
   private final DocRouter.Range range;
@@ -43,6 +45,7 @@ public class Slice extends ZkNodeProps {
   private final Map<String,Replica> replicas;
   private final Replica leader;
   private final String state;
+  private final String parent;
 
   /**
    * @param name  The name of the slice
@@ -75,6 +78,11 @@ public class Slice extends ZkNodeProps {
     }
     **/
 
+    if (propMap.containsKey(PARENT) && propMap.get(PARENT) != null)
+      this.parent = (String) propMap.get(PARENT);
+    else
+      this.parent = null;
+
     replicationFactor = null;  // future
 
     // add the replicas *after* the other properties (for aesthetics, so it's easy to find slice properties in the JSON output)
@@ -150,6 +158,10 @@ public class Slice extends ZkNodeProps {
     return state;
   }
 
+  public String getParent() {
+    return parent;
+  }
+
   @Override
   public String toString() {
     return name + ':' + JSONUtil.toJSON(propMap);

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Mon Oct 21 18:58:24 2013
@@ -120,6 +120,13 @@ public class SolrZkClient {
           });
     } catch (Throwable e) {
       connManager.close();
+      if (keeper != null) {
+        try {
+          keeper.close();
+        } catch (InterruptedException e1) {
+          Thread.currentThread().interrupt();
+        }
+      }
       throw new RuntimeException(e);
     }
     
@@ -127,6 +134,11 @@ public class SolrZkClient {
       connManager.waitForConnected(clientConnectTimeout);
     } catch (Throwable e) {
       connManager.close();
+      try {
+        keeper.close();
+      } catch (InterruptedException e1) {
+        Thread.currentThread().interrupt();
+      }
       throw new RuntimeException(e);
     }
     numOpens.incrementAndGet();

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java Mon Oct 21 18:58:24 2013
@@ -27,7 +27,7 @@ import org.apache.zookeeper.data.ACL;
 
 
 public class ZkCmdExecutor {
-  private long retryDelay = 1300L; // 300 ms over for padding
+  private long retryDelay = 1500L; // 500 ms over for padding
   private int retryCount;
   private List<ACL> acl = ZooDefs.Ids.OPEN_ACL_UNSAFE;
   

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java Mon Oct 21 18:58:24 2013
@@ -58,7 +58,7 @@ public class ZkNodeProps implements JSON
     if ((keyVals.length & 0x01) != 0) {
       throw new IllegalArgumentException("arguments should be key,value");
     }
-    Map<String,Object> propMap = new HashMap<String,Object>(keyVals.length>>1);
+    Map<String,Object> propMap = new LinkedHashMap<String,Object>(keyVals.length>>1);
     for (int i = 0; i < keyVals.length; i+=2) {
       propMap.put(keyVals[i].toString(), keyVals[i+1]);
     }
@@ -125,7 +125,7 @@ public class ZkNodeProps implements JSON
   public Object get(String key) {
     return propMap.get(key);
   }
-  
+
   @Override
   public String toString() {
     return JSONUtil.toJSON(this);

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Mon Oct 21 18:58:24 2013
@@ -59,6 +59,7 @@ public class ZkStateReader {
   public static final String SHARD_ID_PROP = "shard";
   public static final String SHARD_RANGE_PROP = "shard_range";
   public static final String SHARD_STATE_PROP = "shard_state";
+  public static final String SHARD_PARENT_PROP = "shard_parent";
   public static final String NUM_SHARDS_PROP = "numShards";
   public static final String LEADER_PROP = "leader";
   
@@ -218,11 +219,6 @@ public class ZkStateReader {
               Stat stat = new Stat();
               byte[] data = zkClient.getData(CLUSTER_STATE, thisWatch, stat ,
                   true);
-              List<String> liveNodes = zkClient.getChildren(
-                  LIVE_NODES_ZKNODE, this, true);
-     
-              Set<String> liveNodesSet = new HashSet<String>();
-              liveNodesSet.addAll(liveNodes);
               Set<String> ln = ZkStateReader.this.clusterState.getLiveNodes();
               ClusterState clusterState = ClusterState.load(stat.getVersion(), data, ln);
               // update volatile

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java Mon Oct 21 18:58:24 2013
@@ -80,9 +80,15 @@ public abstract class CoreAdminParams
   /** The shard range in solr cloud */
   public final static String SHARD_STATE = "shard.state";
 
+  /** The parent shard if applicable */
+  public final static String SHARD_PARENT = "shard.parent";
+
   /** The target core to which a split index should be written to
    * Multiple targetCores can be specified by multiple targetCore parameters */
   public final static String TARGET_CORE = "targetCore";
+
+  /** The hash ranges to be used to split a shard or an index */
+  public final static String RANGES = "ranges";
   
   public static final String ROLES = "roles";
   

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/params/FacetParams.java Mon Oct 21 18:58:24 2013
@@ -32,6 +32,12 @@ public interface FacetParams {
    */
   public static final String FACET = "facet";
 
+  /**
+   * Numeric option indicating the maximum number of threads to be used
+   * in counting facet field vales 
+   */
+  public static final String FACET_THREADS = FACET + ".threads";
+
   /** What method should be used to do the faceting */
   public static final String FACET_METHOD = FACET + ".method";
 

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java Mon Oct 21 18:58:24 2013
@@ -62,7 +62,6 @@ public class JavaBinCodec {
           END = 15,
 
           SOLRINPUTDOC = 16,
-          SOLRINPUTDOC_CHILDS = 17,
 
           // types that combine tag + length (or other info) in a single byte
           TAG_AND_LEN = (byte) (1 << 5),
@@ -359,35 +358,32 @@ public class JavaBinCodec {
 
   public SolrInputDocument readSolrInputDocument(DataInputInputStream dis) throws IOException {
     int sz = readVInt(dis);
-    dis.readByte(); // skip childDocuments tag
-    int childsSize = readVInt(dis);
     float docBoost = (Float)readVal(dis);
     SolrInputDocument sdoc = new SolrInputDocument();
     sdoc.setDocumentBoost(docBoost);
     for (int i = 0; i < sz; i++) {
       float boost = 1.0f;
       String fieldName;
-      Object boostOrFieldName = readVal(dis);
-      if (boostOrFieldName instanceof Float) {
-        boost = (Float)boostOrFieldName;
+      Object obj = readVal(dis); // could be a boost, a field name, or a child document
+      if (obj instanceof Float) {
+        boost = (Float)obj;
         fieldName = (String)readVal(dis);
+      } else if (obj instanceof SolrInputDocument) {
+        sdoc.addChildDocument((SolrInputDocument)obj);
+        continue;
       } else {
-        fieldName = (String)boostOrFieldName;
+        fieldName = (String)obj;
       }
       Object fieldVal = readVal(dis);
       sdoc.setField(fieldName, fieldVal, boost);
     }
-    for (int i = 0; i < childsSize; i++) {
-      dis.readByte(); // skip solrinputdoc tag
-      SolrInputDocument child = readSolrInputDocument(dis);
-      sdoc.addChildDocument(child);
-    }
     return sdoc;
   }
 
   public void writeSolrInputDocument(SolrInputDocument sdoc) throws IOException {
-    writeTag(SOLRINPUTDOC, sdoc.size());
-    writeTag(SOLRINPUTDOC_CHILDS, sdoc.getChildDocuments().size());    
+    List<SolrInputDocument> children = sdoc.getChildDocuments();
+    int sz = sdoc.size() + (children==null ? 0 : children.size());
+    writeTag(SOLRINPUTDOC, sz);
     writeFloat(sdoc.getDocumentBoost());
     for (SolrInputField inputField : sdoc.values()) {
       if (inputField.getBoost() != 1.0f) {
@@ -396,8 +392,10 @@ public class JavaBinCodec {
       writeExternString(inputField.getName());
       writeVal(inputField.getValue());
     }
-    for (SolrInputDocument child : sdoc.getChildDocuments()) {
-      writeSolrInputDocument(child);
+    if (children != null) {
+      for (SolrInputDocument child : sdoc.getChildDocuments()) {
+        writeSolrInputDocument(child);
+      }
     }
   }
 

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/java/org/apache/solr/common/util/NamedList.java Mon Oct 21 18:58:24 2013
@@ -19,10 +19,14 @@ package org.apache.solr.common.util;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.solr.common.SolrException;
+
 /**
  * A simple container class for modeling an ordered list of name/value pairs.
  *
@@ -257,7 +261,23 @@ public class NamedList<T> implements Clo
     }
     return result;
   }
-
+  
+  /**
+   * Removes all values matching the specified name
+   *
+   * @param name Name
+   */
+  private void killAll(String name) {
+    int sz = size();
+    // Go through the list backwards, removing matches as found.
+    for (int i = sz - 1; i >= 0; i--) {
+      String n = getName(i);
+      if (name==n || (name!=null && name.equals(n))) {
+        remove(i);
+      }
+    }
+  }
+  
   /**
    * Recursively parses the NamedList structure to arrive at a specific element.
    * As you descend the NamedList tree, the last element can be any type,
@@ -470,6 +490,126 @@ public class NamedList<T> implements Clo
     return null;
   }
 
+  /**
+   * Removes and returns all values for the specified name.  Returns null if
+   * no matches found.  This method will return all matching objects,
+   * regardless of data type.  If you are parsing Solr config options, the
+   * {@link #removeConfigArgs(String)} or {@link #removeBooleanArg(String)}
+   * methods will probably work better.
+   *
+   * @param name Name
+   * @return List of values
+   */
+  public List<T> removeAll(String name) {
+    List<T> result = new ArrayList<T>();
+    result = getAll(name);
+    if (result.size() > 0 ) {
+      killAll(name);
+      return result;
+    }
+    return null;
+  }
+
+  /**
+   * Used for getting a boolean argument from a NamedList object.  If the name
+   * is not present, returns null.  If there is more than one value with that
+   * name, or if the value found is not a Boolean or a String, throws an
+   * exception.  If there is only one value present and it is a Boolean or a
+   * String, the value is removed and returned as a Boolean. If an exception
+   * is thrown, the NamedList is not modified. See {@link #removeAll(String)}
+   * and {@link #removeConfigArgs(String)} for additional ways of gathering
+   * configuration information from a NamedList.
+   * 
+   * @param name
+   *          The key to look up in the NamedList.
+   * @return The boolean value found.
+   * @throws SolrException
+   *           If multiple values are found for the name or the value found is
+   *           not a Boolean or a String.
+   */
+  public Boolean removeBooleanArg(final String name) {
+    Boolean bool;
+    List<T> values = getAll(name);
+    if (0 == values.size()) {
+      return null;
+    }
+    if (values.size() > 1) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Only one '" + name + "' is allowed");
+    }
+    Object o = get(name);
+    if (o instanceof Boolean) {
+      bool = (Boolean)o;
+    } else if (o instanceof CharSequence) {
+      bool = Boolean.parseBoolean(o.toString());
+    } else {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "'" + name + "' must have type 'bool' or 'str'; found " + o.getClass());
+    }
+    remove(name);
+    return bool;
+  }
+  
+  /**
+   * Used for getting one or many arguments from NamedList objects that hold
+   * configuration parameters. Finds all entries in the NamedList that match
+   * the given name. If they are all strings or arrays of strings, remove them
+   * from the NamedList and return the individual elements as a {@link Collection}.
+   * Parameter order will be preserved if the returned collection is handled as
+   * an {@link ArrayList}. Throws SolrException if any of the values associated
+   * with the name are not strings or arrays of strings.  If exception is
+   * thrown, the NamedList is not modified.  Returns an empty collection if no
+   * matches found.  If you need to remove and retrieve all matching items from
+   * the NamedList regardless of data type, use {@link #removeAll(String)} instead.
+   * The {@link #removeBooleanArg(String)} method can be used for retrieving a
+   * boolean argument.
+   * 
+   * @param name
+   *          The key to look up in the NamedList.
+   * @return A collection of the values found.
+   * @throws SolrException
+   *           If values are found for the input key that are not strings or
+   *           arrays of strings.
+   */
+  @SuppressWarnings("rawtypes")
+  public Collection<String> removeConfigArgs(final String name)
+      throws SolrException {
+    List<T> objects = getAll(name);
+    List<String> collection = new ArrayList<String>(size() / 2);
+    final String err = "init arg '" + name + "' must be a string "
+        + "(ie: 'str'), or an array (ie: 'arr') containing strings; found: ";
+    
+    for (Object o : objects) {
+      if (o instanceof String) {
+        collection.add((String) o);
+        continue;
+      }
+      
+      // If it's an array, convert to List (which is a Collection).
+      if (o instanceof Object[]) {
+        o = Arrays.asList((Object[]) o);
+      }
+      
+      // If it's a Collection, collect each value.
+      if (o instanceof Collection) {
+        for (Object item : (Collection) o) {
+          if (!(item instanceof String)) {
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, err + item.getClass());
+          }
+          collection.add((String) item);
+        }
+        continue;
+      }
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, err + o.getClass());
+    }
+    
+    if (collection.size() > 0) {
+      killAll(name);
+    }
+    
+    return collection;
+  }
+  
   public void clear() {
     nvPairs.clear();
   }

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/solr/solr.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/solr/solr.xml?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/solr/solr.xml (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test-files/solrj/solr/solr.xml Mon Oct 21 18:58:24 2013
@@ -29,6 +29,7 @@
     If 'null' (or absent), cores will not be manageable via request handler
   -->
   <cores adminPath="/admin/cores" defaultCoreName="collection1" host="127.0.0.1" hostPort="${hostPort:8983}" hostContext="${hostContext:solr}" zkClientTimeout="8000" genericCoreNodeNames="${genericCoreNodeNames:true}">
-    <core name="collection1" instanceDir="collection1" shard="${shard:}" collection="${collection:collection1}" config="${solrconfig:solrconfig.xml}" schema="${schema:schema.xml}"/>
+    <core name="collection1" instanceDir="collection1" shard="${shard:}" collection="${collection:collection1}" />
+    <!--config="${solrconfig:solrconfig.xml}" schema="${schema:schema.xml}"/-->
   </cores>
 </solr>

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/MultiCoreExampleTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/MultiCoreExampleTestBase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/MultiCoreExampleTestBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/MultiCoreExampleTestBase.java Mon Oct 21 18:58:24 2013
@@ -46,11 +46,6 @@ public abstract class MultiCoreExampleTe
 
   @Override public String getSolrHome() { return ExternalPaths.EXAMPLE_MULTICORE_HOME; }
 
-  protected void setupCoreContainer() {
-    cores = new CoreContainer();
-    cores.load();
-  }
-  
   @Override public void setUp() throws Exception {
     super.setUp();
 
@@ -65,11 +60,6 @@ public abstract class MultiCoreExampleTe
     System.setProperty( "solr.core0.data.dir", this.dataDir1.getCanonicalPath() ); 
     System.setProperty( "solr.core1.data.dir", this.dataDir2.getCanonicalPath() );
 
-    setupCoreContainer();
-
-    SolrCore.log.info("CORES=" + cores + " : " + cores.getCoreNames());
-
-
   }
   
   @Override
@@ -84,8 +74,6 @@ public abstract class MultiCoreExampleTe
         System.err.println("!!!! WARNING: best effort to remove " + dataDir2.getAbsolutePath() + " FAILED !!!!!");
       }
     }
-
-    cores.shutdown();
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java Mon Oct 21 18:58:24 2013
@@ -73,7 +73,7 @@ import org.slf4j.LoggerFactory;
  *
  * @since solr 1.3
  */
-abstract public class SolrExampleTests extends SolrJettyTestBase
+abstract public class SolrExampleTests extends SolrExampleTestsBase
 {
   private static Logger log = LoggerFactory.getLogger(SolrExampleTests.class);
 
@@ -395,130 +395,6 @@ abstract public class SolrExampleTests e
     }
   }
 
-  /**
-   * query the example
-   */
- @Test
- public void testCommitWithinOnAdd() throws Exception
-  {    
-    // make sure it is empty...
-    SolrServer server = getSolrServer();
-    server.deleteByQuery( "*:*" );// delete everything!
-    server.commit();
-    QueryResponse rsp = server.query( new SolrQuery( "*:*") );
-    Assert.assertEquals( 0, rsp.getResults().getNumFound() );
-
-    // Now try a timed commit...
-    SolrInputDocument doc3 = new SolrInputDocument();
-    doc3.addField( "id", "id3", 1.0f );
-    doc3.addField( "name", "doc3", 1.0f );
-    doc3.addField( "price", 10 );
-    UpdateRequest up = new UpdateRequest();
-    up.add( doc3 );
-    up.setCommitWithin( 500 );  // a smaller commitWithin caused failures on the following assert
-    up.process( server );
-    
-    rsp = server.query( new SolrQuery( "*:*") );
-    Assert.assertEquals( 0, rsp.getResults().getNumFound() );
-    
-    // TODO: not a great way to test this - timing is easily out
-    // of whack due to parallel tests and various computer specs/load
-    Thread.sleep( 1000 ); // wait 1 sec
-
-    // now check that it comes out...
-    rsp = server.query( new SolrQuery( "id:id3") );
-    
-    int cnt = 0;
-    while (rsp.getResults().getNumFound() == 0) {
-      // wait and try again for slower/busier machines
-      // and/or parallel test effects.
-      
-      if (cnt++ == 10) {
-        break;
-      }
-      
-      Thread.sleep( 2000 ); // wait 2 seconds...
-      
-      rsp = server.query( new SolrQuery( "id:id3") );
-    }
-    
-    Assert.assertEquals( 1, rsp.getResults().getNumFound() );
-    
-    // Now test the new convenience parameter on the add() for commitWithin
-    SolrInputDocument doc4 = new SolrInputDocument();
-    doc4.addField( "id", "id4", 1.0f );
-    doc4.addField( "name", "doc4", 1.0f );
-    doc4.addField( "price", 10 );
-    server.add(doc4, 500);
-    
-    Thread.sleep( 1000 ); // wait 1 sec
-
-    // now check that it comes out...
-    rsp = server.query( new SolrQuery( "id:id4") );
-
-    cnt = 0;
-    while (rsp.getResults().getNumFound() == 0) {
-      // wait and try again for slower/busier machines
-      // and/or parallel test effects.
-      
-      if (cnt++ == 10) {
-        break;
-      }
-      
-      Thread.sleep( 2000 ); // wait 2 seconds...
-      
-      rsp = server.query( new SolrQuery( "id:id3") );
-    }
-    
-    Assert.assertEquals( 1, rsp.getResults().getNumFound() );
-  }
- 
- @Test
- public void testCommitWithinOnDelete() throws Exception
-  {    
-    // make sure it is empty...
-    SolrServer server = getSolrServer();
-    server.deleteByQuery( "*:*" );// delete everything!
-    server.commit();
-    QueryResponse rsp = server.query( new SolrQuery( "*:*") );
-    Assert.assertEquals( 0, rsp.getResults().getNumFound() );
-
-    // Now add one document...
-    SolrInputDocument doc3 = new SolrInputDocument();
-    doc3.addField( "id", "id3", 1.0f );
-    doc3.addField( "name", "doc3", 1.0f );
-    doc3.addField( "price", 10 );
-    server.add(doc3);
-    server.commit();
-
-    // now check that it comes out...
-    rsp = server.query( new SolrQuery( "id:id3") );    
-    Assert.assertEquals( 1, rsp.getResults().getNumFound() );
-    
-    // now test commitWithin on a delete
-    UpdateRequest up = new UpdateRequest();
-    up.setCommitWithin(1000);
-    up.deleteById("id3");
-    up.process( server );
-    
-    // the document should still be there
-    rsp = server.query( new SolrQuery( "id:id3") );
-    Assert.assertEquals( 1, rsp.getResults().getNumFound() );
-    
-    // check if the doc has been deleted every 250 ms for 30 seconds
-    long timeout = System.currentTimeMillis() + 30000;
-    do {
-      Thread.sleep( 250 ); // wait 250 ms
-      
-      rsp = server.query( new SolrQuery( "id:id3") );
-      if(rsp.getResults().getNumFound()==0) {
-        return;
-      }
-    } while(System.currentTimeMillis()<timeout);
-    
-    Assert.fail("commitWithin failed to commit");
-  }
-
   @Test
   public void testErrorHandling() throws Exception
   {    
@@ -708,64 +584,6 @@ abstract public class SolrExampleTests e
     rsp = server.query( new SolrQuery( "*:*") );
     Assert.assertEquals( 5 , rsp.getResults().getNumFound() );
   }
-
-
- protected void assertNumFound( String query, int num ) throws SolrServerException, IOException
-  {
-    QueryResponse rsp = getSolrServer().query( new SolrQuery( query ) );
-    if( num != rsp.getResults().getNumFound() ) {
-      fail( "expected: "+num +" but had: "+rsp.getResults().getNumFound() + " :: " + rsp.getResults() );
-    }
-  }
-
- @Test
- public void testAddDelete() throws Exception
-  {    
-    SolrServer server = getSolrServer();
-    
-    // Empty the database...
-    server.deleteByQuery( "*:*" );// delete everything!
-    
-    SolrInputDocument[] doc = new SolrInputDocument[3];
-    for( int i=0; i<3; i++ ) {
-      doc[i] = new SolrInputDocument();
-      doc[i].setField( "id", i + " & 222", 1.0f );
-    }
-    String id = (String) doc[0].getField( "id" ).getFirstValue();
-    
-    server.add( doc[0] );
-    server.commit();
-    assertNumFound( "*:*", 1 ); // make sure it got in
-    
-    // make sure it got in there
-    server.deleteById( id );
-    server.commit();
-    assertNumFound( "*:*", 0 ); // make sure it got out
-    
-    // add it back 
-    server.add( doc[0] );
-    server.commit();
-    assertNumFound( "*:*", 1 ); // make sure it got in
-    server.deleteByQuery( "id:\""+ClientUtils.escapeQueryChars(id)+"\"" );
-    server.commit();
-    assertNumFound( "*:*", 0 ); // make sure it got out
-    
-    // Add two documents
-    for( SolrInputDocument d : doc ) {
-      server.add( d );
-    }
-    server.commit();
-    assertNumFound( "*:*", 3 ); // make sure it got in
-    
-    // should be able to handle multiple delete commands in a single go
-    List<String> ids = new ArrayList<String>();
-    for( SolrInputDocument d : doc ) {
-      ids.add(d.getFieldValue("id").toString());
-    }
-    server.deleteById(ids);
-    server.commit();
-    assertNumFound( "*:*", 0 ); // make sure it got out
-  }
   
  @Test
  public void testLukeHandler() throws Exception
@@ -1225,54 +1043,6 @@ abstract public class SolrExampleTests e
   }
 
   @Test
-  public void testStreamingRequest() throws Exception {
-    SolrServer server = getSolrServer();
-    // Empty the database...
-    server.deleteByQuery( "*:*" );// delete everything!
-    server.commit();
-    assertNumFound( "*:*", 0 ); // make sure it got in
-   
-    // Add some docs to the index
-    UpdateRequest req = new UpdateRequest();
-    for( int i=0; i<10; i++ ) {
-      SolrInputDocument doc = new SolrInputDocument();
-      doc.addField("id", "" + i );
-      doc.addField("cat", "foocat");
-      req.add( doc );
-    }
-    req.setAction(ACTION.COMMIT, true, true );
-    req.process( server );
-   
-    // Make sure it ran OK
-    SolrQuery query = new SolrQuery("*:*");
-    query.set( CommonParams.FL, "id,score,_docid_" );
-    QueryResponse response = server.query(query);
-    assertEquals(0, response.getStatus());
-    assertEquals(10, response.getResults().getNumFound());
-   
-    // Now make sure each document gets output
-    final AtomicInteger cnt = new AtomicInteger( 0 );
-    server.queryAndStreamResponse(query, new StreamingResponseCallback() {
-
-      @Override
-      public void streamDocListInfo(long numFound, long start, Float maxScore) {
-        assertEquals(10, numFound );
-      }
-
-      @Override
-      public void streamSolrDocument(SolrDocument doc) {
-        cnt.incrementAndGet();
-        
-        // Make sure the transformer works for streaming
-        Float score = (Float)doc.get( "score" );
-        assertEquals( "should have score", new Float(1.0), score );
-      }
-     
-    });
-    assertEquals(10, cnt.get() );
-  }
-
-  @Test
   public void testChineseDefaults() throws Exception {
     SolrServer server = getSolrServer();
     // Empty the database...

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExceptionTest.java Mon Oct 21 18:58:24 2013
@@ -42,6 +42,7 @@ public class SolrExceptionTest extends L
       SolrServer client = new HttpSolrServer("http://[ff01::114]:11235/solr/", httpClient);
       SolrQuery query = new SolrQuery("test123");
       client.query(query);
+      client.shutdown();
     } catch (SolrServerException sse) {
       gotExpectedError = true;
       /***

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java Mon Oct 21 18:58:24 2013
@@ -38,6 +38,7 @@ import org.apache.solr.client.solrj.impl
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.impl.LBHttpSolrServer;
 import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.SolrResponseBase;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -107,9 +108,14 @@ public class TestLBHttpSolrServer extend
       docs.add(doc);
     }
     HttpSolrServer solrServer = new HttpSolrServer(solrInstance.getUrl(), httpClient);
-    UpdateResponse resp = solrServer.add(docs);
-    assertEquals(0, resp.getStatus());
-    resp = solrServer.commit();
+    SolrResponseBase resp;
+    try {
+      resp = solrServer.add(docs);
+      assertEquals(0, resp.getStatus());
+      resp = solrServer.commit();
+    } finally {
+      solrServer.shutdown();
+    }
     assertEquals(0, resp.getStatus());
   }
 
@@ -273,6 +279,11 @@ public class TestLBHttpSolrServer extend
       return "solrj/solr/collection1/conf/solrconfig-slave1.xml";
     }
 
+    public String getSolrXmlFile() {
+      return "solrj/solr/solr.xml";
+    }
+
+
     public void setUp() throws Exception {
       File home = new File(LuceneTestCase.TEMP_DIR,
               getClass().getName() + "-" + System.currentTimeMillis());
@@ -286,6 +297,8 @@ public class TestLBHttpSolrServer extend
       dataDir.mkdirs();
       confDir.mkdirs();
 
+      FileUtils.copyFile(SolrTestCaseJ4.getFile(getSolrXmlFile()), new File(homeDir, "solr.xml"));
+
       File f = new File(confDir, "solrconfig.xml");
       FileUtils.copyFile(SolrTestCaseJ4.getFile(getSolrConfigFile()), f);
       f = new File(confDir, "schema.xml");

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/MultiCoreEmbeddedTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/MultiCoreEmbeddedTest.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/MultiCoreEmbeddedTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/MultiCoreEmbeddedTest.java Mon Oct 21 18:58:24 2013
@@ -19,6 +19,8 @@ package org.apache.solr.client.solrj.emb
 
 import org.apache.solr.client.solrj.MultiCoreExampleTestBase;
 import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
 
 /**
  * This runs SolrServer test using 
@@ -33,6 +35,19 @@ public class MultiCoreEmbeddedTest exten
     // TODO: fix this test to use MockDirectoryFactory
     System.clearProperty("solr.directoryFactory");
     super.setUp();
+    setupCoreContainer();
+    SolrCore.log.info("CORES=" + cores + " : " + cores.getCoreNames());
+  }
+  
+  protected void setupCoreContainer() {
+    cores = new CoreContainer();
+    cores.load();
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    cores.shutdown();
+    super.tearDown();
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/BasicHttpSolrServerTest.java Mon Oct 21 18:58:24 2013
@@ -479,6 +479,7 @@ public class BasicHttpSolrServerTest ext
       server.setDefaultMaxConnectionsPerHost(1);
       fail("Operation should not succeed.");
     } catch (UnsupportedOperationException e) {}
+    server.shutdown();
     client.getConnectionManager().shutdown();
   }
 

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrServerTest.java Mon Oct 21 18:58:24 2013
@@ -18,20 +18,29 @@ package org.apache.solr.client.solrj.imp
  */
 
 import java.io.File;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
+import java.net.MalformedURLException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
 import org.apache.solr.cloud.AbstractZkTestCase;
+import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.util.ExternalPaths;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
+
 /**
  * This test would be faster if we simulated the zk state instead.
  */
@@ -100,57 +109,110 @@ public class CloudSolrServerTest extends
 
     del("*:*");
 
-    indexr(id, 0, "a_t", "to come to the aid of their country.");
+    commit();
     
-    CloudJettyRunner shard1Leader = shardToLeaderJetty.get("shard1");
-    CloudJettyRunner shard2Leader = shardToLeaderJetty.get("shard2");
+    SolrInputDocument doc1 = new SolrInputDocument();
+    doc1.addField(id, "0");
+    doc1.addField("a_t", "hello1");
+    SolrInputDocument doc2 = new SolrInputDocument();
+    doc2.addField(id, "2");
+    doc2.addField("a_t", "hello2");
+    
+    UpdateRequest request = new UpdateRequest();
+    request.add(doc1);
+    request.add(doc2);
+    request.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false);
+    
+    // Test single threaded routed updates for UpdateRequest
+    NamedList response = cloudClient.request(request);
+    CloudSolrServer.RouteResponse rr = (CloudSolrServer.RouteResponse) response;
+    Map<String,LBHttpSolrServer.Req> routes = rr.getRoutes();
+    Iterator<Map.Entry<String,LBHttpSolrServer.Req>> it = routes.entrySet()
+        .iterator();
+    while (it.hasNext()) {
+      Map.Entry<String,LBHttpSolrServer.Req> entry = it.next();
+      String url = entry.getKey();
+      UpdateRequest updateRequest = (UpdateRequest) entry.getValue()
+          .getRequest();
+      SolrInputDocument doc = updateRequest.getDocuments().get(0);
+      String id = doc.getField("id").getValue().toString();
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.add("q", "id:" + id);
+      params.add("distrib", "false");
+      QueryRequest queryRequest = new QueryRequest(params);
+      HttpSolrServer solrServer = new HttpSolrServer(url);
+      QueryResponse queryResponse = queryRequest.process(solrServer);
+      SolrDocumentList docList = queryResponse.getResults();
+      assertTrue(docList.getNumFound() == 1);
+    }
     
-    if (cloudClient.isUpdatesToLeaders()) {
-      // compare leaders list
-      assertEquals(2, cloudClient.getLeaderUrlLists().get("collection1").size());
-      HashSet<String> leaderUrlSet = new HashSet<String>();
-      leaderUrlSet.addAll(cloudClient.getLeaderUrlLists().get("collection1"));
-      assertTrue("fail check for leader:" + shard1Leader.url + " in "
-          + leaderUrlSet, leaderUrlSet.contains(shard1Leader.url + "/"));
-      assertTrue("fail check for leader:" + shard2Leader.url + " in "
-          + leaderUrlSet, leaderUrlSet.contains(shard2Leader.url + "/"));
-      
-      // compare replicas list
-      Set<String> replicas = new HashSet<String>();
-      List<CloudJettyRunner> jetties = shardToJetty.get("shard1");
-      for (CloudJettyRunner cjetty : jetties) {
-        replicas.add(cjetty.url);
-      }
-      jetties = shardToJetty.get("shard2");
-      for (CloudJettyRunner cjetty : jetties) {
-        replicas.add(cjetty.url);
-      }
-      replicas.remove(shard1Leader.url);
-      replicas.remove(shard2Leader.url);
-      
-      assertEquals(replicas.size(),
-          cloudClient.getReplicasLists().get("collection1").size());
-      
-      for (String url : cloudClient.getReplicasLists().get("collection1")) {
-        assertTrue("fail check for replica:" + url + " in " + replicas,
-            replicas.contains(stripTrailingSlash(url)));
+    // Test the deleteById routing for UpdateRequest
+    
+    UpdateRequest delRequest = new UpdateRequest();
+    delRequest.deleteById("0");
+    delRequest.deleteById("2");
+    delRequest.setAction(AbstractUpdateRequest.ACTION.COMMIT, false, false);
+    cloudClient.request(delRequest);
+    ModifiableSolrParams qParams = new ModifiableSolrParams();
+    qParams.add("q", "*:*");
+    QueryRequest qRequest = new QueryRequest(qParams);
+    QueryResponse qResponse = qRequest.process(cloudClient);
+    SolrDocumentList docs = qResponse.getResults();
+    assertTrue(docs.getNumFound() == 0);
+    
+    // Test Multi-Threaded routed updates for UpdateRequest
+    
+    CloudSolrServer threadedClient = null;
+    try {
+      threadedClient = new CloudSolrServer(zkServer.getZkAddress());
+      threadedClient.setParallelUpdates(true);
+      threadedClient.setDefaultCollection("collection1");
+      response = threadedClient.request(request);
+      rr = (CloudSolrServer.RouteResponse) response;
+      routes = rr.getRoutes();
+      it = routes.entrySet()
+          .iterator();
+      while (it.hasNext()) {
+        Map.Entry<String,LBHttpSolrServer.Req> entry = it.next();
+        String url = entry.getKey();
+        UpdateRequest updateRequest = (UpdateRequest) entry.getValue()
+            .getRequest();
+        SolrInputDocument doc = updateRequest.getDocuments().get(0);
+        String id = doc.getField("id").getValue().toString();
+        ModifiableSolrParams params = new ModifiableSolrParams();
+        params.add("q", "id:" + id);
+        params.add("distrib", "false");
+        QueryRequest queryRequest = new QueryRequest(params);
+        HttpSolrServer solrServer = new HttpSolrServer(url);
+        QueryResponse queryResponse = queryRequest.process(solrServer);
+        SolrDocumentList docList = queryResponse.getResults();
+        assertTrue(docList.getNumFound() == 1);
       }
+    } finally {
+      threadedClient.shutdown();
     }
     
+    del("*:*");
+    commit();
   }
-
-  private String stripTrailingSlash(String url) {
-    if (url.endsWith("/")) {
-      return url.substring(0, url.length() - 1);
-    }
-    return url;
-  }
-  
   
   @Override
   protected void indexr(Object... fields) throws Exception {
     SolrInputDocument doc = getDoc(fields);
     indexDoc(doc);
   }
+  
+  public void testShutdown() throws MalformedURLException {
+    CloudSolrServer server = new CloudSolrServer("[ff01::114]:33332");
+    try {
+      server.setZkConnectTimeout(100);
+      server.connect();
+      fail("Expected exception");
+    } catch (RuntimeException e) {
+      assertTrue(e.getCause() instanceof TimeoutException);
+    } finally {
+      server.shutdown();
+    }
+  }
 
 }

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestUpdateRequestCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestUpdateRequestCodec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestUpdateRequestCodec.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestUpdateRequestCodec.java Mon Oct 21 18:58:24 2013
@@ -16,23 +16,22 @@
  */
 package org.apache.solr.client.solrj.request;
 
-import junit.framework.Assert;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.SolrInputField;
-import org.apache.solr.common.util.FastInputStream;
-import org.junit.Test;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.util.Set;
-import java.util.HashSet;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.ArrayList;
+import java.util.Set;
+
+import junit.framework.Assert;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.junit.Test;
 
 /**
  * Test for UpdateRequestCodec
@@ -93,7 +92,7 @@ public class TestUpdateRequestCodec exte
     };
 
     UpdateRequest updateUnmarshalled = codec.unmarshal(new ByteArrayInputStream(baos.toByteArray()) ,handler);
-    Assert.assertNull(updateUnmarshalled.getDocuments());
+
     for (SolrInputDocument document : docs) {
       updateUnmarshalled.add(document);
     }
@@ -144,7 +143,7 @@ public class TestUpdateRequestCodec exte
     };
 
     UpdateRequest updateUnmarshalled = codec.unmarshal(new ByteArrayInputStream(baos.toByteArray()) ,handler);
-    Assert.assertNull(updateUnmarshalled.getDocuments());
+ 
     for (SolrInputDocument document : docs) {
       updateUnmarshalled.add(document);
     }

Modified: lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java (original)
+++ lucene/dev/branches/lucene4956/solr/solrj/src/test/org/apache/solr/common/util/NamedListTest.java Mon Oct 21 18:58:24 2013
@@ -17,7 +17,11 @@
 
 package org.apache.solr.common.util;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.SolrException;
 
 public class NamedListTest extends LuceneTestCase {
   public void testRemove() {
@@ -25,9 +29,71 @@ public class NamedListTest extends Lucen
     nl.add("key1", "value1");
     nl.add("key2", "value2");
     assertEquals(2, nl.size());
-    String value = nl.remove(0);
+    String value = null;
+    value = nl.remove(0);
     assertEquals("value1", value);
     assertEquals(1, nl.size());
+    value = nl.remove("key2");
+    assertEquals("value2", value);
+    assertEquals(0, nl.size());
+  }
+  
+  public void testRemoveAll() {
+    NamedList<String> nl = new NamedList<String>();
+    nl.add("key1", "value1-1");
+    nl.add("key2", "value2-1");
+    nl.add("key1", "value1-2");
+    nl.add("key2", "value2-2");
+    nl.add("key1", "value1-3");
+    nl.add("key2", "value2-3");
+    nl.add("key1", "value1-4");
+    nl.add("key2", "value2-4");
+    nl.add("key1", "value1-5");
+    nl.add("key2", "value2-5");
+    nl.add("key1", "value1-6");
+    assertEquals(11, nl.size());
+    List<String> values = null;
+    values = nl.removeAll("key1");
+    assertEquals("value1-1", values.get(0));
+    assertEquals("value1-3", values.get(2));
+    assertEquals(6, values.size());
+    assertEquals(5, nl.size());
+    values = nl.removeAll("key2");
+    assertEquals(5, values.size());
+    assertEquals(0, nl.size());
+  }
+  
+  public void testRemoveArgs() {
+    NamedList<Object> nl = new NamedList<Object>();
+    nl.add("key1", "value1-1");
+    nl.add("key2", "value2-1");
+    nl.add("key1", "value1-2");
+    nl.add("key2", "value2-2");
+    nl.add("key1", "value1-3");
+    nl.add("key2", "value2-3");
+    nl.add("key1", "value1-4");
+    nl.add("key2", "value2-4");
+    nl.add("key1", "value1-5");
+    nl.add("key2", "value2-5");
+    nl.add("key1", "value1-6");
+    nl.add("key2", 0);
+    nl.add("key2", "value2-7");
+    assertEquals(13, nl.size());
+    List<String> values = (ArrayList<String>) nl.removeConfigArgs("key1");
+    assertEquals("value1-1", values.get(0));
+    assertEquals("value1-3", values.get(2));
+    assertEquals(6, values.size());
+    assertEquals(7, nl.size());
+    try {
+      values = (ArrayList<String>) nl.removeConfigArgs("key2");
+      fail();
+    }
+    catch(SolrException e) {
+      // Expected exception.
+      assertTrue(true);
+    }
+    // nl should be unmodified when removeArgs throws an exception.
+    assertEquals(7, nl.size());
   }
   
   public void testRecursive() {

Modified: lucene/dev/branches/lucene4956/solr/test-framework/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/build.xml?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/build.xml (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/build.xml Mon Oct 21 18:58:24 2013
@@ -70,14 +70,27 @@
       <solr-jarify basedir="${javadoc.dir}/${name}" destfile="${build.dir}/${final.name}-javadoc.jar"/>
     </sequential>
   </target>
-
   <target name="module-jars-to-solr"
-          depends="jar-test-framework">
+          depends="-module-jars-to-solr-not-for-package,-module-jars-to-solr-package"/>
+  <target name="-module-jars-to-solr-not-for-package" unless="called.from.create-package">
+    <antcall target="jar-test-framework" inheritall="true"/>
+    <property name="test-framework.uptodate" value="true"/>
     <mkdir dir="${build.dir}/lucene-libs"/>
     <copy todir="${build.dir}/lucene-libs" preservelastmodified="true" flatten="true" failonerror="true" overwrite="true">
       <path refid="solr.test.framework.lucene.libs" />
     </copy>
   </target>
+  <target name="-module-jars-to-solr-package" if="called.from.create-package">
+    <antcall target="-unpack-lucene-tgz" inheritall="true"/>
+    <pathconvert property="relative.solr.test.framework.lucene.libs" pathsep=",">
+      <path refid="solr.test.framework.lucene.libs"/>
+      <globmapper from="${common.build.dir}/*" to="*" handledirsep="true"/>
+    </pathconvert>
+    <mkdir dir="${build.dir}/lucene-libs"/>
+    <copy todir="${build.dir}/lucene-libs" preservelastmodified="true" flatten="true" failonerror="true" overwrite="true">
+      <fileset dir="${lucene.tgz.unpack.dir}/lucene-${version}" includes="${relative.solr.test.framework.lucene.libs}"/>
+    </copy>
+  </target>
 
   <target name="dist" depends="module-jars-to-solr, common-solr.dist">
     <!-- we're not a contrib, our lucene-libs and go in a special place -->

Modified: lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/ivy.xml Mon Oct 21 18:58:24 2013
@@ -32,13 +32,13 @@
     </configurations>
 
     <dependencies defaultconf="default">
-      <dependency org="org.apache.ant" name="ant" rev="1.8.2" transitive="false" />
+      <dependency org="org.apache.ant" name="ant" rev="${/org.apache.ant/ant}" transitive="false" />
 
-      <dependency org="junit" name="junit" rev="4.10" transitive="false" conf="default->*;junit4-stdalone->*">
+      <dependency org="junit" name="junit" rev="${/junit/junit}" transitive="false" conf="default->*;junit4-stdalone->*">
         <exclude org="org.hamcrest" module="hamcrest-core"/>
       </dependency>
-      <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="2.0.10" transitive="false" conf="default->*;junit4-stdalone->*" />
-      <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="2.0.10" transitive="false" conf="default->*;junit4-stdalone->*" />
+      <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="${/com.carrotsearch.randomizedtesting/junit4-ant}" transitive="false" conf="default->*;junit4-stdalone->*" />
+      <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="${/com.carrotsearch.randomizedtesting/randomizedtesting-runner}" transitive="false" conf="default->*;junit4-stdalone->*" />
 
       <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
     </dependencies>

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java Mon Oct 21 18:58:24 2013
@@ -18,13 +18,13 @@ package org.apache.solr;
  */
 
 import java.io.File;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.SortedMap;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner.SSLConfig;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.util.ExternalPaths;
 import org.eclipse.jetty.servlet.ServletHolder;
@@ -38,25 +38,19 @@ abstract public class SolrJettyTestBase 
 {
   private static Logger log = LoggerFactory.getLogger(SolrJettyTestBase.class);
 
-  // Try not introduce a dependency on the example schema or config unless you need to.
-  // using configs in the test directory allows more flexibility to change "example"
-  // without breaking configs.
-  public String getSolrHome() { return ExternalPaths.EXAMPLE_HOME; }
-
-  private static boolean manageSslProps = true;
   private static File TEST_KEYSTORE;
-  private static final Map<String,String> SSL_PROPS = new HashMap<String,String>();
   static {
     TEST_KEYSTORE = (null == ExternalPaths.SOURCE_HOME)
       ? null : new File(ExternalPaths.SOURCE_HOME, "example/etc/solrtest.keystore");
-    String keystorePath = null == TEST_KEYSTORE ? null : TEST_KEYSTORE.getAbsolutePath();
+  }
 
-    SSL_PROPS.put("tests.jettySsl","false");
-    SSL_PROPS.put("tests.jettySsl.clientAuth","false");
-    SSL_PROPS.put("javax.net.ssl.keyStore", keystorePath);
-    SSL_PROPS.put("javax.net.ssl.keyStorePassword","secret");
-    SSL_PROPS.put("javax.net.ssl.trustStore", keystorePath);
-    SSL_PROPS.put("javax.net.ssl.trustStorePassword","secret");
+  private static void initSSLConfig(SSLConfig sslConfig, String keystorePath) {
+    sslConfig.useSsl = false;
+    sslConfig.clientAuth = false;
+    sslConfig.keyStore = keystorePath;
+    sslConfig.keyStorePassword = "secret";
+    sslConfig.trustStore = keystorePath;
+    sslConfig.trustStorePassword = "secret";
   }
 
   /**
@@ -73,46 +67,43 @@ abstract public class SolrJettyTestBase 
   @BeforeClass
   public static void beforeSolrJettyTestBase() throws Exception {
 
-    // consume the same amount of random no matter what
-    final boolean trySsl = random().nextBoolean();
-    final boolean trySslClientAuth = random().nextBoolean();
+
     
     // only randomize SSL if we are a solr test with access to the example keystore
     if (null == getExampleKeystoreFile()) {
       log.info("Solr's example keystore not defined (not a solr test?) skipping SSL randomization");
-      manageSslProps = false;
       return;
     }
 
     assertTrue("test keystore does not exist, randomized ssl testing broken: " +
                getExampleKeystoreFile().getAbsolutePath(), 
                getExampleKeystoreFile().exists() );
-    
-    // only randomize SSL if none of the SSL_PROPS are already set
-    final Map<Object,Object> sysprops = System.getProperties();
-    for (String prop : SSL_PROPS.keySet()) {
-      if (sysprops.containsKey(prop)) {
-        log.info("System property explicitly set, so skipping randomized ssl properties: " + prop);
-        manageSslProps = false;
-        return;
-      }
-    }
 
-    if (manageSslProps) {
-      log.info("Randomized ssl ({}) and clientAuth ({})", trySsl, trySslClientAuth);
-      for (String prop : SSL_PROPS.keySet()) {
-        System.setProperty(prop, SSL_PROPS.get(prop));
-      }
-      // now explicitly re-set the two random values
-      System.setProperty("tests.jettySsl", String.valueOf(trySsl));
-      System.setProperty("tests.jettySsl.clientAuth", String.valueOf(trySslClientAuth));
-    }
   }
 
   public static JettySolrRunner jetty;
   public static int port;
   public static SolrServer server = null;
   public static String context;
+  
+  public static SSLConfig getSSLConfig() {
+    SSLConfig sslConfig = new SSLConfig();
+    
+    final boolean trySsl = random().nextBoolean();
+    final boolean trySslClientAuth = random().nextBoolean();
+    
+    log.info("Randomized ssl ({}) and clientAuth ({})", trySsl,
+        trySslClientAuth);
+    String keystorePath = null == TEST_KEYSTORE ? null : TEST_KEYSTORE
+        .getAbsolutePath();
+    initSSLConfig(sslConfig, keystorePath);
+    
+    sslConfig.useSsl = trySsl;
+    sslConfig.clientAuth = trySslClientAuth;
+    
+    initSSLConfig(sslConfig, keystorePath);
+    return sslConfig;
+  }
 
   public static JettySolrRunner createJetty(String solrHome, String configFile, String schemaFile, String context,
                                             boolean stopAtShutdown, SortedMap<ServletHolder,String> extraServlets) 
@@ -127,7 +118,7 @@ abstract public class SolrJettyTestBase 
 
     context = context==null ? "/solr" : context;
     SolrJettyTestBase.context = context;
-    jetty = new JettySolrRunner(solrHome, context, 0, configFile, schemaFile, stopAtShutdown, extraServlets);
+    jetty = new JettySolrRunner(solrHome, context, 0, configFile, schemaFile, stopAtShutdown, extraServlets, getSSLConfig());
 
     jetty.start();
     port = jetty.getLocalPort();
@@ -147,11 +138,6 @@ abstract public class SolrJettyTestBase 
       jetty = null;
     }
     server = null;
-    if (manageSslProps) {
-      for (String prop : SSL_PROPS.keySet()) {
-        System.clearProperty(prop);
-      }
-    }
   }
 
 
@@ -188,4 +174,21 @@ abstract public class SolrJettyTestBase 
       return new EmbeddedSolrServer( h.getCoreContainer(), "" );
     }
   }
+
+  // Sets up the necessary config files for Jetty. At least some tests require that the solrconfig from the test
+  // file directory are used, but some also require that the solr.xml file be explicitly there as of SOLR-4817
+  public static void setupJettyTestHome(File solrHome, String collection) throws Exception {
+    if (solrHome.exists()) {
+      FileUtils.deleteDirectory(solrHome);
+    }
+    copySolrHomeToTemp(solrHome, collection);
+  }
+
+  public static void cleanUpJettyHome(File solrHome) throws Exception {
+    if (solrHome.exists()) {
+      FileUtils.deleteDirectory(solrHome);
+    }
+  }
+
+
 }

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java Mon Oct 21 18:58:24 2013
@@ -20,6 +20,7 @@ package org.apache.solr;
 import com.carrotsearch.randomizedtesting.RandomizedContext;
 import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import org.apache.commons.codec.Charsets;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -1161,6 +1162,32 @@ public abstract class SolrTestCaseJ4 ext
     }
   }
 
+  public static class IValsPercent extends IVals {
+    final int[] percentAndValue;
+    public IValsPercent(int... percentAndValue) {
+      this.percentAndValue = percentAndValue;
+    }
+
+    @Override
+    public int getInt() {
+      int r = between(0,99);
+      int cumulative = 0;
+      for (int i=0; i<percentAndValue.length; i+=2) {
+        cumulative += percentAndValue[i];
+        if (r < cumulative) {
+          return percentAndValue[i+1];
+        }
+      }
+
+      return percentAndValue[percentAndValue.length-1];
+    }
+
+    @Override
+    public Comparable get() {
+      return getInt();
+    }
+  }
+
   public static class FVal extends Vals {
     final float min;
     final float max;
@@ -1279,14 +1306,14 @@ public abstract class SolrTestCaseJ4 ext
 
   protected class FldType {
     public String fname;
-    public IRange numValues;
+    public IVals numValues;
     public Vals vals;
 
     public FldType(String fname, Vals vals) {
       this(fname, ZERO_ONE, vals);
     }
 
-    public FldType(String fname, IRange numValues, Vals vals) {
+    public FldType(String fname, IVals numValues, Vals vals) {
       this.fname = fname;
       this.numValues = numValues;
       this.vals = vals;      
@@ -1598,17 +1625,63 @@ public abstract class SolrTestCaseJ4 ext
       throw new RuntimeException("XPath is invalid", e2);
     }
   }
-  // Creates a mininmal conf dir.
-  public void copyMinConf(File dstRoot) throws IOException {
+  public static void copyMinConf(File dstRoot) throws IOException {
+    copyMinConf(dstRoot, null);
+  }
+
+  // Creates a minimal conf dir. Optionally adding in a core.properties file from the string passed in
+  // the string to write to the core.properties file may be null in which case nothing is done with it.
+  // propertiesContent may be an empty string, which will actually work.
+  public static void copyMinConf(File dstRoot, String propertiesContent) throws IOException {
 
     File subHome = new File(dstRoot, "conf");
-    assertTrue("Failed to make subdirectory ", dstRoot.mkdirs());
+    if (! dstRoot.exists()) {
+      assertTrue("Failed to make subdirectory ", dstRoot.mkdirs());
+    }
+    if (propertiesContent != null) {
+      FileUtils.writeStringToFile(new File(dstRoot, "core.properties"), propertiesContent, Charsets.UTF_8.toString());
+    }
     String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
     FileUtils.copyFile(new File(top, "schema-tiny.xml"), new File(subHome, "schema.xml"));
     FileUtils.copyFile(new File(top, "solrconfig-minimal.xml"), new File(subHome, "solrconfig.xml"));
     FileUtils.copyFile(new File(top, "solrconfig.snippet.randomindexconfig.xml"), new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
   }
 
+  // Creates minimal full setup, including the old solr.xml file that used to be hard coded in ConfigSolrXmlOld
+  // TODO: remove for 5.0
+  public static void copyMinFullSetup(File dstRoot) throws IOException {
+    if (! dstRoot.exists()) {
+      assertTrue("Failed to make subdirectory ", dstRoot.mkdirs());
+    }
+    File xmlF = new File(SolrTestCaseJ4.TEST_HOME(), "solr.xml");
+    FileUtils.copyFile(xmlF, new File(dstRoot, "solr.xml"));
+    copyMinConf(dstRoot);
+  }
+
+  // Creates a consistent configuration, _including_ solr.xml at dstRoot. Creates collection1/conf and copies
+  // the stock files in there. Seems to be indicated for some tests when we remove the default, hard-coded
+  // solr.xml from being automatically synthesized from SolrConfigXmlOld.DEFAULT_SOLR_XML.
+  public static void copySolrHomeToTemp(File dstRoot, String collection) throws IOException {
+    if (!dstRoot.exists()) {
+      assertTrue("Failed to make subdirectory ", dstRoot.mkdirs());
+    }
+
+    FileUtils.copyFile(new File(SolrTestCaseJ4.TEST_HOME(), "solr.xml"), new File(dstRoot, "solr.xml"));
+
+    File subHome = new File(dstRoot, collection + File.separator + "conf");
+    String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
+    FileUtils.copyFile(new File(top, "currency.xml"), new File(subHome, "currency.xml"));
+    FileUtils.copyFile(new File(top, "mapping-ISOLatin1Accent.txt"), new File(subHome, "mapping-ISOLatin1Accent.txt"));
+    FileUtils.copyFile(new File(top, "old_synonyms.txt"), new File(subHome, "old_synonyms.txt"));
+    FileUtils.copyFile(new File(top, "open-exchange-rates.json"), new File(subHome, "open-exchange-rates.json"));
+    FileUtils.copyFile(new File(top, "protwords.txt"), new File(subHome, "protwords.txt"));
+    FileUtils.copyFile(new File(top, "schema.xml"), new File(subHome, "schema.xml"));
+    FileUtils.copyFile(new File(top, "solrconfig.snippet.randomindexconfig.xml"), new File(subHome, "solrconfig.snippet.randomindexconfig.xml"));
+    FileUtils.copyFile(new File(top, "solrconfig.xml"), new File(subHome, "solrconfig.xml"));
+    FileUtils.copyFile(new File(top, "stopwords.txt"), new File(subHome, "stopwords.txt"));
+    FileUtils.copyFile(new File(top, "synonyms.txt"), new File(subHome, "synonyms.txt"));
+  }
+
   public static CoreDescriptorBuilder buildCoreDescriptor(CoreContainer container, String name, String instancedir) {
     return new CoreDescriptorBuilder(container, name, instancedir);
   }

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java Mon Oct 21 18:58:24 2013
@@ -148,7 +148,7 @@ public abstract class AbstractDistribZkT
       for (Map.Entry<String,Slice> entry : slices.entrySet()) {
         Map<String,Replica> shards = entry.getValue().getReplicasMap();
         for (Map.Entry<String,Replica> shard : shards.entrySet()) {
-          if (verbose) System.out.println("rstate:"
+          if (verbose) System.out.println("replica:" + shard.getValue().getName() + " rstate:"
               + shard.getValue().getStr(ZkStateReader.STATE_PROP)
               + " live:"
               + clusterState.liveNodesContain(shard.getValue().getNodeName()));

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Mon Oct 21 18:58:24 2013
@@ -17,8 +17,10 @@ package org.apache.solr.cloud;
  * limitations under the License.
  */
 
+import org.apache.commons.io.FilenameUtils;
 import org.apache.http.params.CoreConnectionPNames;
 import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServer;
@@ -44,6 +46,7 @@ import org.apache.solr.common.params.Col
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.SolrResourceLoader;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -118,6 +121,7 @@ public abstract class AbstractFullDistri
   protected Map<String,CloudJettyRunner> shardToLeaderJetty = new HashMap<String,CloudJettyRunner>();
   private boolean cloudInit;
   protected boolean checkCreatedVsState;
+  protected boolean useJettyDataDir = true;
   
   public static class CloudJettyRunner {
     public JettySolrRunner jetty;
@@ -243,6 +247,7 @@ public abstract class AbstractFullDistri
   protected CloudSolrServer createCloudClient(String defaultCollection)
       throws MalformedURLException {
     CloudSolrServer server = new CloudSolrServer(zkServer.getZkAddress(), random().nextBoolean());
+    server.setParallelUpdates(random().nextBoolean());
     if (defaultCollection != null) server.setDefaultCollection(defaultCollection);
     server.getLbServer().getHttpClient().getParams()
         .setParameter(CoreConnectionPNames.CONNECTION_TIMEOUT, 5000);
@@ -342,8 +347,8 @@ public abstract class AbstractFullDistri
           getClass().getName() + "-jetty" + cnt + "-" + System.currentTimeMillis());
       jettyDir.mkdirs();
       setupJettySolrHome(jettyDir);
-      JettySolrRunner j = createJetty(jettyDir, getDataDir(testDir + "/jetty"
-          + cnt), null, "solrconfig.xml", null);
+      JettySolrRunner j = createJetty(jettyDir, useJettyDataDir ? getDataDir(testDir + "/jetty"
+          + cnt) : null, null, "solrconfig.xml", null);
       jettys.add(j);
       SolrServer client = createNewSolrServer(j.getLocalPort());
       clients.add(client);
@@ -455,14 +460,45 @@ public abstract class AbstractFullDistri
   }
   
   public JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride) throws Exception {
-
-    JettySolrRunner jetty = new JettySolrRunner(solrHome.getAbsolutePath(), context, 0, solrConfigOverride, schemaOverride, false, getExtraServlets());
+    // randomly test a relative solr.home path
+    if (random().nextBoolean()) {
+      solrHome = getRelativeSolrHomePath(solrHome);
+    }
+    
+    JettySolrRunner jetty = new JettySolrRunner(solrHome.getPath(), context, 0, solrConfigOverride, schemaOverride, false, getExtraServlets());
     jetty.setShards(shardList);
     jetty.setDataDir(getDataDir(dataDir));
     jetty.start();
     
     return jetty;
   }
+
+  private File getRelativeSolrHomePath(File solrHome) {
+    String path = SolrResourceLoader.normalizeDir(new File(".").getAbsolutePath());
+    String base = new File(solrHome.getPath()).getAbsolutePath();
+    
+    if (base.startsWith("."));
+    base.replaceFirst("\\.", new File(".").getName());
+    
+    if (path.endsWith(File.separator + ".")) {
+      path = path.substring(0, path.length() - 2);
+    }
+    
+    int splits = path.split("\\" + File.separator).length;
+    
+    StringBuilder p = new StringBuilder();
+    for (int i = 0; i < splits - 2; i++) {
+      p.append(".." + File.separator);
+    }   
+    
+    String prefix = FilenameUtils.getPrefix(path);
+    if (base.startsWith(prefix)) {
+      base = base.substring(prefix.length());
+    }
+
+    solrHome = new File(p.toString() + base);
+    return solrHome;
+  }
   
   protected void updateMappingsFromZk(List<JettySolrRunner> jettys,
       List<SolrServer> clients) throws Exception {
@@ -1502,7 +1538,10 @@ public abstract class AbstractFullDistri
     createCollection(null, collectionName, numShards, numReplicas, maxShardsPerNode, null, null);
   }
 
-  protected void createCollection(Map<String,List<Integer>> collectionInfos, String collectionName, Map<String,Object> collectionProps, SolrServer client )  throws SolrServerException, IOException{
+  protected void createCollection(Map<String,List<Integer>> collectionInfos, String collectionName, Map<String,Object> collectionProps, SolrServer client)  throws SolrServerException, IOException{
+    createCollection(collectionInfos, collectionName, collectionProps, client, null);
+  }
+  protected void createCollection(Map<String,List<Integer>> collectionInfos, String collectionName, Map<String,Object> collectionProps, SolrServer client, String confSetName)  throws SolrServerException, IOException{
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("action", CollectionAction.CREATE.toString());
     for (Map.Entry<String, Object> entry : collectionProps.entrySet()) {
@@ -1517,7 +1556,11 @@ public abstract class AbstractFullDistri
     if(numShards==null){
       numShards = (Integer) OverseerCollectionProcessor.COLL_PROPS.get(REPLICATION_FACTOR);
     }
-
+    
+    if (confSetName != null) {
+      params.set("collection.configName", confSetName);
+    }
+    
     int clientIndex = random().nextInt(2);
     List<Integer> list = new ArrayList<Integer>();
     list.add(numShards);
@@ -1549,13 +1592,25 @@ public abstract class AbstractFullDistri
       String collectionName, int numShards, int numReplicas, int maxShardsPerNode, SolrServer client, String createNodeSetStr) throws SolrServerException, IOException {
 
     createCollection(collectionInfos, collectionName,
-        OverseerCollectionProcessor.asMap(
+        ZkNodeProps.makeMap(
         NUM_SLICES, numShards,
         REPLICATION_FACTOR, numReplicas,
         CREATE_NODE_SET, createNodeSetStr,
         MAX_SHARDS_PER_NODE, maxShardsPerNode),
         client);
   }
+  
+  protected void createCollection(Map<String,List<Integer>> collectionInfos,
+      String collectionName, int numShards, int numReplicas, int maxShardsPerNode, SolrServer client, String createNodeSetStr, String configName) throws SolrServerException, IOException {
+
+    createCollection(collectionInfos, collectionName,
+        ZkNodeProps.makeMap(
+        NUM_SLICES, numShards,
+        REPLICATION_FACTOR, numReplicas,
+        CREATE_NODE_SET, createNodeSetStr,
+        MAX_SHARDS_PER_NODE, maxShardsPerNode),
+        client, configName);
+  }
 
   @Override
   protected SolrServer createNewSolrServer(int port) {
@@ -1662,6 +1717,7 @@ public abstract class AbstractFullDistri
       synchronized(this) {
         try {
           commondCloudSolrServer = new CloudSolrServer(zkServer.getZkAddress(), random().nextBoolean());
+          commondCloudSolrServer.setParallelUpdates(random().nextBoolean());
           commondCloudSolrServer.setDefaultCollection(DEFAULT_COLLECTION);
           commondCloudSolrServer.connect();
         } catch (MalformedURLException e) {

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/AbstractZkTestCase.java Mon Oct 21 18:58:24 2013
@@ -102,27 +102,26 @@ public abstract class AbstractZkTestCase
     zkClient.makePath("/collections/control_collection/shards", CreateMode.PERSISTENT, true);
 
     // for now, always upload the config and schema to the canonical names
-    putConfig(zkClient, solrhome, config, "solrconfig.xml");
-    putConfig(zkClient, solrhome, schema, "schema.xml");
+    putConfig("conf1", zkClient, solrhome, config, "solrconfig.xml");
+    putConfig("conf1", zkClient, solrhome, schema, "schema.xml");
 
-    putConfig(zkClient, solrhome, "solrconfig.snippet.randomindexconfig.xml");
-    putConfig(zkClient, solrhome, "stopwords.txt");
-    putConfig(zkClient, solrhome, "protwords.txt");
-    putConfig(zkClient, solrhome, "currency.xml");
-    putConfig(zkClient, solrhome, "open-exchange-rates.json");
-    putConfig(zkClient, solrhome, "mapping-ISOLatin1Accent.txt");
-    putConfig(zkClient, solrhome, "old_synonyms.txt");
-    putConfig(zkClient, solrhome, "synonyms.txt");
-    
+    putConfig("conf1", zkClient, solrhome, "solrconfig.snippet.randomindexconfig.xml");
+    putConfig("conf1", zkClient, solrhome, "stopwords.txt");
+    putConfig("conf1", zkClient, solrhome, "protwords.txt");
+    putConfig("conf1", zkClient, solrhome, "currency.xml");
+    putConfig("conf1", zkClient, solrhome, "open-exchange-rates.json");
+    putConfig("conf1", zkClient, solrhome, "mapping-ISOLatin1Accent.txt");
+    putConfig("conf1", zkClient, solrhome, "old_synonyms.txt");
+    putConfig("conf1", zkClient, solrhome, "synonyms.txt");
     zkClient.close();
   }
 
-  private static void putConfig(SolrZkClient zkClient, File solrhome, final String name)
+  public static void putConfig(String confName, SolrZkClient zkClient, File solrhome, final String name)
       throws Exception {
-    putConfig(zkClient, solrhome, name, name);
+    putConfig(confName, zkClient, solrhome, name, name);
   }
 
-  private static void putConfig(SolrZkClient zkClient, File solrhome, final String srcName, String destName)
+  public static void putConfig(String confName, SolrZkClient zkClient, File solrhome, final String srcName, String destName)
       throws Exception {
     File file = new File(solrhome, "collection1"
         + File.separator + "conf" + File.separator + srcName);
@@ -131,7 +130,7 @@ public abstract class AbstractZkTestCase
       return;
     }
 
-    String destPath = "/configs/conf1/" + destName;
+    String destPath = "/configs/" + confName + "/" + destName;
     log.info("put " + file.getAbsolutePath() + " to " + destPath);
     zkClient.makePath(destPath, file, false, true);
   }

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java Mon Oct 21 18:58:24 2013
@@ -153,11 +153,11 @@ public class ChaosMonkey {
     }
   }
   
-  private void causeConnectionLoss(JettySolrRunner jetty) {
+  public static void causeConnectionLoss(JettySolrRunner jetty) {
     causeConnectionLoss(jetty, ZkTestServer.TICK_TIME * 2 + 200);
   }
   
-  private void causeConnectionLoss(JettySolrRunner jetty, int pauseTime) {
+  public static void causeConnectionLoss(JettySolrRunner jetty, int pauseTime) {
     SolrDispatchFilter solrDispatchFilter = (SolrDispatchFilter) jetty
         .getDispatchFilter().getFilter();
     if (solrDispatchFilter != null) {

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java Mon Oct 21 18:58:24 2013
@@ -350,4 +350,8 @@ public class ZkTestServer {
   public void setTheTickTime(int theTickTime) {
     this.theTickTime = theTickTime;
   }
+
+  public String getZkDir() {
+    return zkDir;
+  }
 }

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/ExternalPaths.java Mon Oct 21 18:58:24 2013
@@ -43,6 +43,8 @@ public class ExternalPaths {
   /* @see #SOURCE_HOME */
   public static String EXAMPLE_HOME = new File(SOURCE_HOME, "example/solr").getAbsolutePath();
   /* @see #SOURCE_HOME */
+  public static String EXAMPLE_SCHEMALESS_HOME = new File(SOURCE_HOME, "example/example-schemaless/solr").getAbsolutePath();
+  /* @see #SOURCE_HOME */
   public static String EXAMPLE_MULTICORE_HOME = new File(SOURCE_HOME, "example/multicore").getAbsolutePath();
   /* @see #SOURCE_HOME */
   public static String EXAMPLE_SCHEMA=EXAMPLE_HOME+"/collection1/conf/schema.xml";

Modified: lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java (original)
+++ lucene/dev/branches/lucene4956/solr/test-framework/src/java/org/apache/solr/util/RandomMergePolicy.java Mon Oct 21 18:58:24 2013
@@ -21,6 +21,9 @@ import org.apache.lucene.index.*;
 import org.apache.lucene.index.MergePolicy.MergeSpecification;
 import org.apache.lucene.util.LuceneTestCase;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.Map;
 import java.io.IOException;
 
@@ -32,6 +35,8 @@ import java.io.IOException;
  */
 public final class RandomMergePolicy extends MergePolicy {
 
+  public static Logger log = LoggerFactory.getLogger(RandomMergePolicy.class);
+  
   /** 
    * Not private so tests can inspect it, 
    * Not final so it can be set on clone
@@ -45,6 +50,8 @@ public final class RandomMergePolicy ext
     super(inner.getNoCFSRatio(), 
           (long) (inner.getMaxCFSSegmentSizeMB() * 1024 * 1024));
     this.inner = inner;
+    log.info("RandomMergePolicy wrapping {}: {}",
+             inner.getClass(), inner);
   }
 
   public RandomMergePolicy clone() {