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

svn commit: r1512909 [30/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-too...

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Sun Aug 11 12:19:13 2013
@@ -17,16 +17,6 @@ package org.apache.solr.update.processor
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
@@ -74,6 +64,16 @@ import org.apache.solr.update.VersionInf
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
 // NOT mt-safe... create a new processor for each add thread
@@ -139,7 +139,7 @@ public class DistributedUpdateProcessor 
   // method in this update processor
   private boolean isLeader = true;
   private boolean forwardToLeader = false;
-  private boolean forwardToSubShard = false;
+  private boolean isSubShardLeader = false;
   private List<Node> nodes;
 
   private int numNodes;
@@ -222,9 +222,20 @@ public class DistributedUpdateProcessor 
         // Replica leader = slice.getLeader();
         Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
             collection, shardId);
-        ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(leaderReplica);
-        String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
-        isLeader = coreNodeName.equals(leaderReplica.getName());
+        isLeader = leaderReplica.getName().equals(
+            req.getCore().getCoreDescriptor().getCloudDescriptor()
+                .getCoreNodeName());
+
+        if (!isLeader) {
+          isSubShardLeader = amISubShardLeader(coll, slice, id, doc);
+          if (isSubShardLeader) {
+            String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+            slice = coll.getSlice(myShardId);
+            shardId = myShardId;
+            leaderReplica = zkController.getZkStateReader().getLeaderRetry(collection, myShardId);
+            List<ZkCoreNodeProps> myReplicas = zkController.getZkStateReader().getReplicaProps(collection, shardId, leaderReplica.getName(), coreName, null, ZkStateReader.DOWN);
+          }
+        }
 
         DistribPhase phase =
             DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
@@ -232,18 +243,17 @@ public class DistributedUpdateProcessor 
         doDefensiveChecks(phase);
 
 
-        if (DistribPhase.FROMLEADER == phase) {
+        if (DistribPhase.FROMLEADER == phase && !isSubShardLeader) {
           // we are coming from the leader, just go local - add no urls
           forwardToLeader = false;
-        } else if (isLeader) {
+        } else if (isLeader || isSubShardLeader) {
           // that means I want to forward onto my replicas...
           // so get the replicas...
           forwardToLeader = false;
           List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
-              .getReplicaProps(collection, shardId, coreNodeName,
+              .getReplicaProps(collection, shardId, leaderReplica.getName(),
                   coreName, null, ZkStateReader.DOWN);
 
-          nodes = addSubShardLeaders(coll, shardId, id, doc, nodes);
           if (replicaProps != null) {
             if (nodes == null)  {
             nodes = new ArrayList<Node>(replicaProps.size());
@@ -273,7 +283,7 @@ public class DistributedUpdateProcessor 
         } else {
           // I need to forward onto the leader...
           nodes = new ArrayList<Node>(1);
-          nodes.add(new RetryNode(leaderProps, zkController.getZkStateReader(), collection, shardId));
+          nodes.add(new RetryNode(new ZkCoreNodeProps(leaderReplica), zkController.getZkStateReader(), collection, shardId));
           forwardToLeader = true;
         }
 
@@ -287,8 +297,34 @@ public class DistributedUpdateProcessor 
     return nodes;
   }
 
-  private List<Node> addSubShardLeaders(DocCollection coll, String shardId, String docId, SolrInputDocument doc, List<Node> nodes) {
+  private boolean amISubShardLeader(DocCollection coll, Slice parentSlice, String id, SolrInputDocument doc) throws InterruptedException {
+    // Am I the leader of a shard in "construction" state?
+    String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+    Slice mySlice = coll.getSlice(myShardId);
+    if (Slice.CONSTRUCTION.equals(mySlice.getState())) {
+      Replica myLeader = zkController.getZkStateReader().getLeaderRetry(collection, myShardId);
+      boolean amILeader = myLeader.getName().equals(
+          req.getCore().getCoreDescriptor().getCloudDescriptor()
+              .getCoreNodeName());
+      if (amILeader) {
+        // Does the document belong to my hash range as well?
+        DocRouter.Range myRange = mySlice.getRange();
+        if (myRange == null) myRange = new DocRouter.Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
+        if (parentSlice != null)  {
+          boolean isSubset = parentSlice.getRange() != null && myRange.isSubsetOf(parentSlice.getRange());
+          return isSubset && coll.getRouter().isTargetSlice(id, doc, req.getParams(), myShardId, coll);
+        } else  {
+          // delete by query case -- as long as I am a sub shard leader we're fine
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private List<Node> getSubShardLeaders(DocCollection coll, String shardId, String docId, SolrInputDocument doc) {
     Collection<Slice> allSlices = coll.getSlices();
+    List<Node> nodes = null;
     for (Slice aslice : allSlices) {
       if (Slice.CONSTRUCTION.equals(aslice.getState()))  {
         DocRouter.Range myRange = coll.getSlice(shardId).getRange();
@@ -303,7 +339,6 @@ public class DistributedUpdateProcessor 
             if (nodes == null) nodes = new ArrayList<Node>();
             ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(sliceLeader);
             nodes.add(new StdNode(nodeProps));
-            forwardToSubShard = true;
           }
         }
       }
@@ -312,7 +347,7 @@ public class DistributedUpdateProcessor 
   }
 
   private void doDefensiveChecks(DistribPhase phase) {
-    boolean isReplayOrPeersync = (updateCommand.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (updateCommand.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     if (isReplayOrPeersync) return;
 
     String from = req.getParams().get("distrib.from");
@@ -341,9 +376,11 @@ public class DistributedUpdateProcessor 
       }
     }
 
-    if (isLeader && !localIsLeader) {
+    if ((isLeader && !localIsLeader) || (isSubShardLeader && !localIsLeader)) {
       log.error("ClusterState says we are the leader, but locally we don't think so");
-      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "ClusterState says we are the leader, but locally we don't think so");
+      throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE,
+          "ClusterState says we are the leader (" + zkController.getBaseUrl()
+              + "/" + req.getCore().getName() + "), but locally we don't think so. Request came from " + from);
     }
   }
 
@@ -356,16 +393,15 @@ public class DistributedUpdateProcessor 
     try {
       Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
           collection, shardId);
-      String leaderCoreNodeName = leaderReplica.getName();
-
-      String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
-      isLeader = coreNodeName.equals(leaderCoreNodeName);
+      isLeader = leaderReplica.getName().equals(
+          req.getCore().getCoreDescriptor().getCloudDescriptor()
+              .getCoreNodeName());
 
       // TODO: what if we are no longer the leader?
 
       forwardToLeader = false;
       List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
-          .getReplicaProps(collection, shardId, coreNodeName,
+          .getReplicaProps(collection, shardId, leaderReplica.getName(),
               req.getCore().getName());
       if (replicaProps != null) {
         nodes = new ArrayList<Node>(replicaProps.size());
@@ -373,8 +409,6 @@ public class DistributedUpdateProcessor 
           nodes.add(new StdNode(props));
         }
       }
-
-      nodes = addSubShardLeaders(zkController.getClusterState().getCollection(collection), shardId, null, null, nodes);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
@@ -395,7 +429,7 @@ public class DistributedUpdateProcessor 
     } else {
       isLeader = getNonZkLeaderAssumption(req);
     }
-    
+
     boolean dropCmd = false;
     if (!forwardToLeader) {
       dropCmd = versionAdd(cmd);
@@ -405,22 +439,35 @@ public class DistributedUpdateProcessor 
       // TODO: do we need to add anything to the response?
       return;
     }
-    
+
+    if (zkEnabled && isLeader && !isSubShardLeader)  {
+      DocCollection coll = zkController.getClusterState().getCollection(collection);
+      List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), cmd.getHashableId(), cmd.getSolrInputDocument());
+      // the list<node> will actually have only one element for an add request
+      if (subShardLeaders != null && !subShardLeaders.isEmpty()) {
+        ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
+        params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
+        params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
+            zkController.getBaseUrl(), req.getCore().getName()));
+        params.set("distrib.from.parent", req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId());
+        for (Node subShardLeader : subShardLeaders) {
+          cmdDistrib.syncAdd(cmd, subShardLeader, params);
+        }
+      }
+    }
+
     ModifiableSolrParams params = null;
     if (nodes != null) {
-      
+
       params = new ModifiableSolrParams(filterParams(req.getParams()));
-      params.set(DISTRIB_UPDATE_PARAM, 
-                 (isLeader ? 
-                  DistribPhase.FROMLEADER.toString() : 
+      params.set(DISTRIB_UPDATE_PARAM,
+                 (isLeader || isSubShardLeader ?
+                  DistribPhase.FROMLEADER.toString() :
                   DistribPhase.TOLEADER.toString()));
-      if (isLeader) {
+      if (isLeader || isSubShardLeader) {
         params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
             zkController.getBaseUrl(), req.getCore().getName()));
       }
-      if (forwardToSubShard)  {
-        params.set("distrib.from.parent", req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId());
-      }
 
       params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
           zkController.getBaseUrl(), req.getCore().getName()));
@@ -559,7 +606,7 @@ public class DistributedUpdateProcessor 
       }
     }
 
-    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     boolean leaderLogic = isLeader && !isReplayOrPeersync;
 
 
@@ -640,7 +687,7 @@ public class DistributedUpdateProcessor 
         if (willDistrib) {
           clonedDoc = cmd.solrDoc.deepCopy();
         }
-        
+
         // TODO: possibly set checkDeleteByQueries as a flag on the command?
         doLocalAdd(cmd);
         
@@ -781,15 +828,30 @@ public class DistributedUpdateProcessor 
       return;
     }
 
+    if (zkEnabled && isLeader && !isSubShardLeader)  {
+      DocCollection coll = zkController.getClusterState().getCollection(collection);
+      List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), null, null);
+      // the list<node> will actually have only one element for an add request
+      if (subShardLeaders != null && !subShardLeaders.isEmpty()) {
+        ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
+        params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
+        params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
+            zkController.getBaseUrl(), req.getCore().getName()));
+        params.set("distrib.from.parent", cloudDesc.getShardId());
+        cmdDistrib.syncDelete(cmd, subShardLeaders, params);
+      }
+    }
+
+
     ModifiableSolrParams params = null;
     if (nodes != null) {
-      
+
       params = new ModifiableSolrParams(filterParams(req.getParams()));
-      params.set(DISTRIB_UPDATE_PARAM, 
-                 (isLeader ? 
-                  DistribPhase.FROMLEADER.toString() : 
+      params.set(DISTRIB_UPDATE_PARAM,
+                 (isLeader || isSubShardLeader ?
+                  DistribPhase.FROMLEADER.toString() :
                   DistribPhase.TOLEADER.toString()));
-      if (isLeader) {
+      if (isLeader || isSubShardLeader) {
         params.set("distrib.from", ZkCoreNodeProps.getCoreUrl(
             zkController.getBaseUrl(), req.getCore().getName()));
       }
@@ -817,9 +879,11 @@ public class DistributedUpdateProcessor 
   }
 
   private void passParam(SolrParams params, ModifiableSolrParams fparams, String param) {
-    String value = params.get(param);
-    if (value != null) {
-      fparams.add(param, value);
+    String[] values = params.getParams(param);
+    if (values != null) {
+      for (String value : values) {
+        fparams.add(param, value);
+      }
     }
   }
 
@@ -843,15 +907,19 @@ public class DistributedUpdateProcessor 
     DistribPhase phase = 
     DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
 
+    DocCollection coll = zkEnabled 
+      ? zkController.getClusterState().getCollection(collection) : null;
+
     if (zkEnabled && DistribPhase.NONE == phase) {
       boolean leaderForAnyShard = false;  // start off by assuming we are not a leader for any shard
 
       ModifiableSolrParams outParams = new ModifiableSolrParams(filterParams(req.getParams()));
       outParams.set(DISTRIB_UPDATE_PARAM, DistribPhase.TOLEADER.toString());
 
-      DocCollection coll = zkController.getClusterState().getCollection(collection);
       SolrParams params = req.getParams();
-      Collection<Slice> slices = coll.getRouter().getSearchSlices(params.get(ShardParams.SHARD_KEYS), params, coll);
+      String route = params.get(ShardParams._ROUTE_);
+      if(route == null) route = params.get(ShardParams.SHARD_KEYS);// deprecated . kept for backcompat
+      Collection<Slice> slices = coll.getRouter().getSearchSlices(route, params, coll);
 
       List<Node> leaders =  new ArrayList<Node>(slices.size());
       for (Slice slice : slices) {
@@ -869,7 +937,7 @@ public class DistributedUpdateProcessor 
         // Am I the leader for this slice?
         ZkCoreNodeProps coreLeaderProps = new ZkCoreNodeProps(leader);
         String leaderCoreNodeName = leader.getName();
-        String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
+        String coreNodeName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
         isLeader = coreNodeName.equals(leaderCoreNodeName);
 
         if (isLeader) {
@@ -917,7 +985,7 @@ public class DistributedUpdateProcessor 
     }
     versionOnUpdate = Math.abs(versionOnUpdate);  // normalize to positive version
 
-    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     boolean leaderLogic = isLeader && !isReplayOrPeersync;
 
     if (!leaderLogic && versionOnUpdate==0) {
@@ -957,16 +1025,53 @@ public class DistributedUpdateProcessor 
       vinfo.unblockUpdates();
     }
 
-
-    // forward to all replicas
-    if (leaderLogic && replicas != null) {
+    if (zkEnabled)  {
+      // forward to all replicas
       ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
       params.set(VERSION_FIELD, Long.toString(cmd.getVersion()));
       params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString());
       params.set("update.from", ZkCoreNodeProps.getCoreUrl(
           zkController.getBaseUrl(), req.getCore().getName()));
-      cmdDistrib.distribDelete(cmd, replicas, params);
-      cmdDistrib.finish();
+
+      boolean someReplicas = false;
+      boolean subShardLeader = false;
+      try {
+        subShardLeader = amISubShardLeader(coll, null, null, null);
+        if (subShardLeader)  {
+          String myShardId = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+          Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(
+              collection, myShardId);
+          List<ZkCoreNodeProps> replicaProps = zkController.getZkStateReader()
+              .getReplicaProps(collection, myShardId, leaderReplica.getName(),
+                  req.getCore().getName(), null, ZkStateReader.DOWN);
+          if (replicaProps != null) {
+            List<Node> myReplicas = new ArrayList<Node>();
+            for (ZkCoreNodeProps replicaProp : replicaProps) {
+              myReplicas.add(new StdNode(replicaProp));
+            }
+            cmdDistrib.distribDelete(cmd, myReplicas, params);
+            someReplicas = true;
+          }
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "", e);
+      }
+
+      if (leaderLogic) {
+        List<Node> subShardLeaders = getSubShardLeaders(coll, cloudDesc.getShardId(), null, null);
+        if (subShardLeaders != null)  {
+          cmdDistrib.syncDelete(cmd, subShardLeaders, params);
+        }
+        if (replicas != null) {
+          cmdDistrib.distribDelete(cmd, replicas, params);
+          someReplicas = true;
+        }
+      }
+
+      if (someReplicas)  {
+        cmdDistrib.finish();
+      }
     }
 
 
@@ -1029,7 +1134,7 @@ public class DistributedUpdateProcessor 
     long signedVersionOnUpdate = versionOnUpdate;
     versionOnUpdate = Math.abs(versionOnUpdate);  // normalize to positive version
 
-    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.REPLAY)) != 0;
+    boolean isReplayOrPeersync = (cmd.getFlags() & (UpdateCommand.REPLAY | UpdateCommand.PEER_SYNC)) != 0;
     boolean leaderLogic = isLeader && !isReplayOrPeersync;
 
     if (!leaderLogic && versionOnUpdate==0) {
@@ -1097,15 +1202,37 @@ public class DistributedUpdateProcessor 
     }
   }
 
-
   @Override
   public void processCommit(CommitUpdateCommand cmd) throws IOException {
     updateCommand = cmd;
-
+    List<Node> nodes = null;
+    boolean singleLeader = false;
     if (zkEnabled) {
       zkCheck();
+      
+      nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
+          .getCloudDescriptor().getCollectionName());
+      if (isLeader && nodes.size() == 1) {
+        singleLeader = true;
+      }
     }
     
+    if (!zkEnabled || req.getParams().getBool(COMMIT_END_POINT, false) || singleLeader) {
+      doLocalCommit(cmd);
+    } else if (zkEnabled) {
+      ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
+      if (!req.getParams().getBool(COMMIT_END_POINT, false)) {
+        params.set(COMMIT_END_POINT, true);
+
+        if (nodes != null) {
+          cmdDistrib.distribCommit(cmd, nodes, params);
+          finish();
+        }
+      }
+    }
+  }
+
+  private void doLocalCommit(CommitUpdateCommand cmd) throws IOException {
     if (vinfo != null) {
       vinfo.lockForUpdate();
     }
@@ -1122,23 +1249,6 @@ public class DistributedUpdateProcessor 
         vinfo.unlockForUpdate();
       }
     }
-    // TODO: we should consider this? commit everyone in the current collection
-
-    if (zkEnabled) {
-      ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
-      if (!req.getParams().getBool(COMMIT_END_POINT, false)) {
-        params.set(COMMIT_END_POINT, true);
-
-        String coreNodeName = zkController.getCoreNodeName(req.getCore().getCoreDescriptor());
-        List<Node> nodes = getCollectionUrls(req, req.getCore().getCoreDescriptor()
-            .getCloudDescriptor().getCollectionName(), coreNodeName);
-
-        if (nodes != null) {
-          cmdDistrib.distribCommit(cmd, nodes, params);
-          finish();
-        }
-      }
-    }
   }
   
   @Override
@@ -1150,7 +1260,7 @@ public class DistributedUpdateProcessor 
  
 
   
-  private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, String coreNodeName) {
+  private List<Node> getCollectionUrls(SolrQueryRequest req, String collection) {
     ClusterState clusterState = req.getCore().getCoreDescriptor()
         .getCoreContainer().getZkController().getClusterState();
     List<Node> urls = new ArrayList<Node>();
@@ -1166,7 +1276,7 @@ public class DistributedUpdateProcessor 
       
       for (Entry<String,Replica> entry : shardMap.entrySet()) {
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
-        if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(coreNodeName)) {
+        if (clusterState.liveNodesContain(nodeProps.getNodeName())) {
           urls.add(new StdNode(nodeProps));
         }
       }

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java Sun Aug 11 12:19:13 2013
@@ -20,11 +20,11 @@ package org.apache.solr.update.processor
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Set;
 import java.util.regex.Pattern;
 
 import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
 import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+import static org.apache.solr.update.processor.FieldMutatingUpdateProcessorFactory.SelectorParams;
 
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
@@ -108,7 +108,7 @@ public abstract class FieldMutatingUpdat
         // for now, don't allow it.
         if (! fname.equals(dest.getName()) ) {
           throw new SolrException(SERVER_ERROR,
-                                  "mutute returned field with different name: " 
+                                  "mutate returned field with different name: " 
                                   + fname + " => " + dest.getName());
         }
         doc.put(dest.getName(), dest);
@@ -118,7 +118,7 @@ public abstract class FieldMutatingUpdat
   }
   
   /**
-   * Interface for idenfifying which fileds should be mutated
+   * Interface for identifying which fields should be mutated
    */
   public static interface FieldNameSelector {
     public boolean shouldMutate(final String fieldName);
@@ -192,79 +192,75 @@ public abstract class FieldMutatingUpdat
   public static FieldNameSelector createFieldNameSelector
     (final SolrResourceLoader loader,
      final SolrCore core,
-     final Set<String> fields,
-     final Set<String> typeNames,
-     final Collection<String> typeClasses,
-     final Collection<Pattern> regexes,
+     final SelectorParams params,
      final FieldNameSelector defSelector) {
-    
-    final Collection<Class> classes 
-      = new ArrayList<Class>(typeClasses.size());
-    
-    for (String t : typeClasses) {
-      try {
-        classes.add(loader.findClass(t, Object.class));
-      } catch (Exception e) {
-        throw new SolrException(SERVER_ERROR,
-                                "Can't resolve typeClass: " + t, e);
-      }
-    }
-    
-    if (classes.isEmpty() && 
-        typeNames.isEmpty() && 
-        regexes.isEmpty() && 
-        fields.isEmpty()) {
+
+    if (params.noSelectorsSpecified()) {
       return defSelector;
     }
     
-    return new ConfigurableFieldNameSelector(core, fields, typeNames, classes, regexes); 
+    return new ConfigurableFieldNameSelector(loader, core, params); 
   }
   
+  
+  
   private static final class ConfigurableFieldNameSelector 
     implements FieldNameSelector {
 
     final SolrCore core;
-    final Set<String> fields;
-    final Set<String> typeNames;
+    final SelectorParams params;
     final Collection<Class> classes;
-    final Collection<Pattern> regexes;
 
-    private ConfigurableFieldNameSelector(final SolrCore core,
-                                          final Set<String> fields,
-                                          final Set<String> typeNames,
-                                          final Collection<Class> classes,
-                                          final Collection<Pattern> regexes) {
+    private ConfigurableFieldNameSelector(final SolrResourceLoader loader,
+                                          final SolrCore core,
+                                          final SelectorParams params) {
       this.core = core;
-      this.fields = fields;
-      this.typeNames = typeNames;
+      this.params = params;
+
+      final Collection<Class> classes = new ArrayList<Class>(params.typeClass.size());
+
+      for (String t : params.typeClass) {
+        try {
+          classes.add(loader.findClass(t, Object.class));
+        } catch (Exception e) {
+          throw new SolrException(SERVER_ERROR, "Can't resolve typeClass: " + t, e);
+        }
+      }
       this.classes = classes;
-      this.regexes = regexes;
     }
 
     @Override
     public boolean shouldMutate(final String fieldName) {
       
-      // order of checks is bsaed on what should be quicker 
+      // order of checks is based on what should be quicker
       // (ie: set lookups faster the looping over instanceOf / matches tests
       
-      if ( ! (fields.isEmpty() || fields.contains(fieldName)) ) {
+      if ( ! (params.fieldName.isEmpty() || params.fieldName.contains(fieldName)) ) {
         return false;
       }
       
       // do not consider it an error if the fieldName has no type
       // there might be another processor dealing with it later
       FieldType t =  core.getLatestSchema().getFieldTypeNoEx(fieldName);
-      if (null != t) {
-        if (! (typeNames.isEmpty() || typeNames.contains(t.getTypeName())) ) {
+      final boolean fieldExists = (null != t);
+
+      if ( (null != params.fieldNameMatchesSchemaField) &&
+           (fieldExists != params.fieldNameMatchesSchemaField) ) {
+        return false;
+      }
+
+      if (fieldExists) { 
+
+        if (! (params.typeName.isEmpty() || params.typeName.contains(t.getTypeName())) ) {
           return false;
         }
         
         if (! (classes.isEmpty() || instanceOfAny(t, classes)) ) {
           return false;
-          }
-      }
+        }
+      } 
       
-      if (! (regexes.isEmpty() || matchesAny(fieldName, regexes)) ) {
+      if (! (params.fieldRegex.isEmpty() || matchesAny(fieldName, params.fieldRegex)) ) {
         return false;
       }
       

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java Sun Aug 11 12:19:13 2013
@@ -58,6 +58,15 @@ import org.apache.solr.util.plugin.SolrC
  * <b>at least one of each</b> to be selected.
  * </p>
  * <p>
+ * The following additional selector may be specified as a &lt;bool&gt; - when specified
+ * as false, only fields that <b>do not</b> match a schema field/dynamic field are selected;
+ * when specified as true, only fields that <b>do</b> match a schema field/dynamic field are
+ * selected:
+ * </p>
+ * <ul>
+ *   <li><code>fieldNameMatchesSchemaField</code> - selecting specific fields based on whether or not they match a schema field</li>
+ * </ul>
+ * <p>
  * One or more <code>excludes</code> &lt;lst&gt; params may also be specified, 
  * containing any of the above criteria, identifying fields to be excluded 
  * from seelction even if they match the selection criteria.  As with the main 
@@ -71,7 +80,7 @@ import org.apache.solr.util.plugin.SolrC
  * fields will be mutated if the name starts with "foo" <i>or</i> "bar"; 
  * <b>unless</b> the field name contains the substring "SKIP" <i>or</i> 
  * the fieldType is (or subclasses) DateField.  Meaning a field named 
- * "foo_SKIP" is gaurunteed not to be selected, but a field named "bar_smith" 
+ * "foo_SKIP" is guaranteed not to be selected, but a field named "bar_smith" 
  * that uses StrField will be selected.
  * </p>
  * <pre class="prettyprint">
@@ -106,6 +115,13 @@ public abstract class FieldMutatingUpdat
     public Set<String> typeName = Collections.emptySet();
     public Collection<String> typeClass = Collections.emptyList();
     public Collection<Pattern> fieldRegex = Collections.emptyList();
+    public Boolean fieldNameMatchesSchemaField = null; // null => not specified
+
+    public boolean noSelectorsSpecified() {
+      return typeClass.isEmpty()  && typeName.isEmpty() 
+          && fieldRegex.isEmpty() && fieldName.isEmpty() 
+          && null == fieldNameMatchesSchemaField;
+    }
   }
 
   private SelectorParams inclusions = new SelectorParams();
@@ -121,7 +137,6 @@ public abstract class FieldMutatingUpdat
                             " inform(SolrCore) never called???");
   }
 
-  @SuppressWarnings("unchecked")
   public static SelectorParams parseSelectorParams(NamedList args) {
     SelectorParams params = new SelectorParams();
     
@@ -145,43 +160,52 @@ public abstract class FieldMutatingUpdat
     // resolve this into actual Class objects later
     params.typeClass = oneOrMany(args, "typeClass");
 
+    // getBooleanArg() returns null if the arg is not specified
+    params.fieldNameMatchesSchemaField = getBooleanArg(args, "fieldNameMatchesSchemaField");
+    
     return params;
   }
-                                                            
-
-  /**
-   * Handles common initialization related to source fields for 
-   * constructoring the FieldNameSelector to be used.
-   *
-   * Will error if any unexpected init args are found, so subclasses should
-   * remove any subclass-specific init args before calling this method.
-   */
-  @SuppressWarnings("unchecked")
-  @Override
-  public void init(NamedList args) {
-
-    inclusions = parseSelectorParams(args);
-
+                               
+  public static Collection<SelectorParams> parseSelectorExclusionParams(NamedList args) {
+    Collection<SelectorParams> exclusions = new ArrayList<SelectorParams>();
     List<Object> excList = args.getAll("exclude");
     for (Object excObj : excList) {
       if (null == excObj) {
         throw new SolrException
-          (SERVER_ERROR, "'exclude' init param can not be null"); 
+            (SERVER_ERROR, "'exclude' init param can not be null");
       }
       if (! (excObj instanceof NamedList) ) {
         throw new SolrException
-          (SERVER_ERROR, "'exclude' init param must be <lst/>"); 
+            (SERVER_ERROR, "'exclude' init param must be <lst/>");
       }
       NamedList exc = (NamedList) excObj;
       exclusions.add(parseSelectorParams(exc));
       if (0 < exc.size()) {
-        throw new SolrException(SERVER_ERROR, 
-                                "Unexpected 'exclude' init sub-param(s): '" + 
-                                args.getName(0) + "'");
+        throw new SolrException(SERVER_ERROR,
+            "Unexpected 'exclude' init sub-param(s): '" +
+                args.getName(0) + "'");
       }
       // call once per instance
       args.remove("exclude");
     }
+    return exclusions;
+  }
+  
+
+  /**
+   * Handles common initialization related to source fields for 
+   * constructing the FieldNameSelector to be used.
+   *
+   * Will error if any unexpected init args are found, so subclasses should
+   * remove any subclass-specific init args before calling this method.
+   */
+  @SuppressWarnings("unchecked")
+  @Override
+  public void init(NamedList args) {
+
+    inclusions = parseSelectorParams(args);
+    exclusions = parseSelectorExclusionParams(args);
+
     if (0 < args.size()) {
       throw new SolrException(SERVER_ERROR, 
                               "Unexpected init param(s): '" + 
@@ -195,25 +219,13 @@ public abstract class FieldMutatingUpdat
     
     selector = 
       FieldMutatingUpdateProcessor.createFieldNameSelector
-      (core.getResourceLoader(),
-       core,
-       inclusions.fieldName,
-       inclusions.typeName,
-       inclusions.typeClass,
-       inclusions.fieldRegex,
-       getDefaultSelector(core));
+          (core.getResourceLoader(), core, inclusions, getDefaultSelector(core));
 
     for (SelectorParams exc : exclusions) {
       selector = FieldMutatingUpdateProcessor.wrap
         (selector,
          FieldMutatingUpdateProcessor.createFieldNameSelector
-         (core.getResourceLoader(),
-          core,
-          exc.fieldName,
-          exc.typeName,
-          exc.typeClass,
-          exc.fieldRegex,
-          FieldMutatingUpdateProcessor.SELECT_NO_FIELDS));
+             (core.getResourceLoader(), core, exc, FieldMutatingUpdateProcessor.SELECT_NO_FIELDS));
     }
   }
   
@@ -270,7 +282,28 @@ public abstract class FieldMutatingUpdat
     return result;
   }
 
+  /**
+   * Removes the first instance of the key from NamedList, returning the Boolean
+   * that key referred to, or null if the key is not specified.
+   * @exception SolrException invalid type or structure
+   */
+  public static Boolean getBooleanArg(final NamedList args, final String key) {
+    Boolean bool;
+    List values = args.getAll(key);
+    if (0 == values.size()) {
+      return null;
+    }
+    if (values.size() > 1) {
+      throw new SolrException(SERVER_ERROR, "Only one '" + key + "' is allowed");
+    }
+    Object o = args.remove(key);
+    if (o instanceof Boolean) {
+      bool = (Boolean)o;
+    } else if (o instanceof CharSequence) {
+      bool = Boolean.parseBoolean(o.toString());
+    } else {
+      throw new SolrException(SERVER_ERROR, "'" + key + "' must have type 'bool' or 'str'; found " + o.getClass());
+    }
+    return bool;
+  }
 }
-
-
-

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/update/processor/UpdateRequestProcessorChain.java Sun Aug 11 12:19:13 2013
@@ -34,23 +34,48 @@ import java.util.ArrayList;
 /**
  * Manages a chain of UpdateRequestProcessorFactories.
  * <p>
- * Chain can be configured via solrconfig.xml:
+ * Chains can be configured via solrconfig.xml using the following syntax...
  * </p>
  * <pre class="prettyprint">
- * &lt;updateRequestProcessors name="key" default="true"&gt;
- *   &lt;processor class="PathToClass1" /&gt;
- *   &lt;processor class="PathToClass2" /&gt;
+ * &lt;updateRequestProcessorChain name="key" default="true"&gt;
+ *   &lt;processor class="package.Class1" /&gt;
+ *   &lt;processor class="package.Class2" &gt;
+ *     &lt;str name="someInitParam1"&gt;value&lt;/str&gt;
+ *     &lt;int name="someInitParam2"&gt;42&lt;/int&gt;
+ *   &lt;/processor&gt;
  *   &lt;processor class="solr.LogUpdateProcessorFactory" &gt;
  *     &lt;int name="maxNumToLog"&gt;100&lt;/int&gt;
  *   &lt;/processor&gt;
  *   &lt;processor class="solr.RunUpdateProcessorFactory" /&gt;
- * &lt;/updateRequestProcessors&gt;
+ * &lt;/updateRequestProcessorChain&gt;
  * </pre>
  * <p>
+ * Multiple Chains can be defined, each with a distinct name.  The name of 
+ * a chain used to handle an update request may be specified using the request 
+ * param <code>update.chain</code>.  If no chain is explicitly selected 
+ * by name, then Solr will attempt to determine a default chain:
+ * </p>
+ * <ul>
+ *  <li>A single configured chain may explicitly be declared with 
+ *      <code>default="true"</code> (see example above)</li>
+ *  <li>If no chain is explicitly declared as the default, Solr will look for
+ *      any chain that does not have a name, and treat it as the default</li>
+ *  <li>As a last resort, Solr will create an implicit default chain 
+ *      consisting of:<ul>
+ *        <li>{@link LogUpdateProcessorFactory}</li>
+ *        <li>{@link DistributedUpdateProcessorFactory}</li>
+ *        <li>{@link RunUpdateProcessorFactory}</li>
+ *      </ul></li>
+ * </ul>
+ *
+ * <p>
  * Allmost all processor chains should end with an instance of 
- * {@link RunUpdateProcessorFactory} unless the user is explicitly 
+ * <code>RunUpdateProcessorFactory</code> unless the user is explicitly 
  * executing the update commands in an alternative custom 
- * <code>UpdateRequestProcessorFactory</code>.
+ * <code>UpdateRequestProcessorFactory</code>.  If a chain includes 
+ * <code>RunUpdateProcessorFactory</code> but does not include a 
+ * <code>DistributingUpdateProcessorFactory</code>, it will be added 
+ * automaticly by {@link #init init()}.
  * </p>
  *
  * @see UpdateRequestProcessorFactory

Modified: lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/java/org/apache/solr/util/PropertiesUtil.java Sun Aug 11 12:19:13 2013
@@ -129,4 +129,24 @@ public class PropertiesUtil {
     }
   }
 
+  /**
+   * Parse the given String value as an integer.  If the string cannot
+   * be parsed, returns the default
+   * @param value    the value to parse
+   * @param defValue the default to return if the value cannot be parsed
+   * @return an integer version of the passed in value
+   */
+  public static Integer toInteger(String value, Integer defValue) {
+    try {
+      return Integer.parseInt(value);
+    }
+    catch (NumberFormatException e) {
+      return defValue;
+    }
+  }
+
+  public static boolean toBoolean(String value) {
+    return "true".equalsIgnoreCase(value) || "on".equalsIgnoreCase(value);
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/log4j.properties Sun Aug 11 12:19:13 2013
@@ -7,3 +7,4 @@ log4j.appender.CONSOLE.layout=org.apache
 log4j.appender.CONSOLE.layout.ConversionPattern=%-5p - %d{yyyy-MM-dd HH:mm:ss.SSS}; %C; %m\n
 
 log4j.logger.org.apache.zookeeper=WARN
+log4j.logger.org.apache.hadoop=WARN

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml Sun Aug 11 12:19:13 2013
@@ -394,6 +394,7 @@
       termVectors="true" termPositions="true" termOffsets="true"/>
    <field name="signatureField" type="string" indexed="true" stored="false"/>
    
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
 
    
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml Sun Aug 11 12:19:13 2013
@@ -221,8 +221,6 @@
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="4" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="byte" class="solr.ByteField" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="short" class="solr.ShortField" omitNorms="true" positionIncrementGap="0"/>
     <fieldtype name="boolean" class="solr.BoolField" sortMissingLast="true"/>
     <fieldtype name="date" class="solr.TrieDateField" precisionStep="0"/>
   </types>
@@ -233,8 +231,6 @@
     <field name="float_f" type="float"/>
     <field name="long_f" type="long"/>
     <field name="double_f" type="double"/>
-    <field name="byte_f" type="byte"/>
-    <field name="short_f" type="short"/>
     <field name="bool_f" type="boolean"/>
     <field name="date_f" type="date"/>
 

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-luceneMatchVersion.xml Sun Aug 11 12:19:13 2013
@@ -17,6 +17,7 @@
 -->
 <schema name="luceneMatchVersionTest" version="1.1">
  <types>
+  <fieldtype name="long" class="solr.TrieLongField"/>
   <fieldtype name="string" class="solr.StrField"/>
   <fieldtype name="text40" class="solr.TextField">
     <analyzer>
@@ -50,6 +51,7 @@
    <field name="textDefault" type="textDefault" indexed="true" stored="false" />
    <field name="textStandardAnalyzer40" type="textStandardAnalyzer40" indexed="true" stored="false" />
    <field name="textStandardAnalyzerDefault" type="textStandardAnalyzerDefault" indexed="true" stored="false" />
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
    <dynamicField name="*_sS" type="string"  indexed="false" stored="true"/>
  </fields>
  <uniqueKey>id</uniqueKey>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml Sun Aug 11 12:19:13 2013
@@ -376,7 +376,7 @@
    <field name="intDefault" type="sint" indexed="true" stored="true" default="42" multiValued="false"/>
    <field name="signatureField" type="string" indexed="true" stored="false"/>
    
-
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
    
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-reversed.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-reversed.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-reversed.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-reversed.xml Sun Aug 11 12:19:13 2013
@@ -25,6 +25,7 @@
   <types>
 
 
+    <fieldtype name="long" class="solr.TrieLongField" />
     <fieldtype name="integer" class="solr.IntField" />
     <fieldtype name="string" class="solr.StrField" />
 
@@ -73,6 +74,8 @@
    <field name="two" type="rev" indexed="true" stored="false"/>
    <field name="three" type="text" indexed="true" stored="false"/>
 
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
+
    <field name="signatureField" type="string" indexed="true" stored="false"/>
    <dynamicField name="*_sS" type="string"  indexed="false" stored="true"/>
 

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml Sun Aug 11 12:19:13 2013
@@ -282,6 +282,8 @@
 
    <field name="tdate" type="tdate" indexed="true" stored="true" />
 
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
+
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.
         RESTRICTION: the glob-like pattern in the name attribute must have

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema-xinclude.xml Sun Aug 11 12:19:13 2013
@@ -1,4 +1,7 @@
 <?xml version="1.0" encoding="UTF-8" ?>
+<!DOCTYPE schema [
+        <!ENTITY schema_entity_include SYSTEM "schema-snippet-types.incl">
+]>
 <!--
  Licensed to the Apache Software Foundation (ASF) under one or more
  contributor license agreements.  See the NOTICE file distributed with
@@ -19,6 +22,7 @@
  <types>
   <fieldType name="string" class="solr.StrField"/>
   <xi:include href="schema-snippet-type.xml" />
+  &schema_entity_include;
  </types>
  <fields>
    <xi:include href="schema-snippet-field.xml" />

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema.xml Sun Aug 11 12:19:13 2013
@@ -48,8 +48,6 @@
     <fieldType name="float" class="solr.TrieFloatField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
     <fieldType name="double" class="solr.TrieDoubleField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="byte" class="solr.ByteField" omitNorms="true" positionIncrementGap="0"/>
-    <fieldType name="short" class="solr.ShortField" omitNorms="true" positionIncrementGap="0"/>
 
 
     <fieldType name="tint" class="solr.TrieIntField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
@@ -583,6 +581,7 @@
    <field name="store" type="location" indexed="true" stored="true" omitNorms="false"/>
 
    <field name="lower" type="lowertok" indexed="false" stored="true" multiValued="true" />
+   <field name="_route_" type="string" indexed="true" stored="true" multiValued="false" />
 
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.
@@ -609,10 +608,6 @@
    <dynamicField name="*_dt" type="date"    indexed="true"  stored="true"/>
    <dynamicField name="*_dt1" type="date"    indexed="true"  stored="true" multiValued="false"/>
    <dynamicField name="*_bcd" type="bcdstr" indexed="true"  stored="true"/>
-   <dynamicField name="*_by"  type="byte"  indexed="true" stored="true"/>
-   <dynamicField name="*_by1" type="byte"  indexed="true" stored="true" multiValued="false"/>
-   <dynamicField name="*_sh" type="short"  indexed="true" stored="true"/>
-   <dynamicField name="*_sh1" type="short"  indexed="true" stored="true" multiValued="false"/>
 
 
       <!-- some trie-coded dynamic fields for faster range queries -->

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema11.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema11.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema11.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema11.xml Sun Aug 11 12:19:13 2013
@@ -322,6 +322,8 @@ valued. -->
 
    <field name="text_no_analyzer" type="text_no_analyzer" indexed="true" />
 
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
+
    <!-- Dynamic field definitions.  If a field name is not found, dynamicFields
         will be used if the name matches any of the patterns.
         RESTRICTION: the glob-like pattern in the name attribute must have

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema12.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema12.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema12.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schema12.xml Sun Aug 11 12:19:13 2013
@@ -420,6 +420,9 @@
    <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
     <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
     <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- sub-centimeter accuracy for RPT; distance calcs -->
+    <fieldType name="location_rpt" class="solr.SpatialRecursivePrefixTreeFieldType"
+      geo="true" distErrPct="0.025" maxDistErr="0.00000009" units="degrees" />
 
   <fieldType name="currency" class="solr.CurrencyField" currencyConfig="currency.xml" multiValued="false" />
  </types>
@@ -517,6 +520,7 @@
    <field name="pointD" type="xyd" indexed="true" stored="true" multiValued="false"/>
    <field name="point_hash" type="geohash" indexed="true" stored="true" multiValued="false"/>
    <field name="store" type="location" indexed="true" stored="true"/>
+   <field name="store_rpt" type="location_rpt" indexed="true" stored="false" />
    
    <!-- Test currency field -->
    <field name="amount_c" type="currency" indexed="true" stored="true" multiValued="false"/>
@@ -609,5 +613,6 @@
    <copyField source="subject" dest="text"/>
 
    <copyField source="foo_copysource_*" dest="bar_copydest_*" />
+   <copyField source="store" dest="store_rpt" />
  
 </schema>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml Sun Aug 11 12:19:13 2013
@@ -539,6 +539,8 @@
    <field name="uniq3" type="string" indexed="true" stored="true"/>
    <field name="nouniq" type="string" indexed="true" stored="true" multiValued="true"/>
 
+   <field name="_version_" type="long" indexed="true" stored="true" multiValued="false" />
+
    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false"/>
 
 

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-SOLR-749.xml Sun Aug 11 12:19:13 2013
@@ -19,6 +19,7 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <queryParser name="foo" class="FooQParserPlugin"/>
   <!-- override the default "lucene" qparser -->

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-altdirectory.xml Sun Aug 11 12:19:13 2013
@@ -19,6 +19,7 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestFSDirectoryFactory"></directoryFactory>
   <indexReaderFactory name="IndexReaderFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestIndexReaderFactory"></indexReaderFactory >

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-basic.xml Sun Aug 11 12:19:13 2013
@@ -22,6 +22,8 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
-  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <requestHandler name="standard" class="solr.StandardRequestHandler" />
+  <requestHandler name="/update" class="solr.UpdateRequestHandler" />
 </config>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-caching.xml Sun Aug 11 12:19:13 2013
@@ -16,6 +16,7 @@
   -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <query>
     <cache name="lfuCacheDecayFalse"
@@ -35,4 +36,4 @@
            size="10"
            initialSize="9" />
   </query>
-</config>
\ No newline at end of file
+</config>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-components-name.xml Sun Aug 11 12:19:13 2013
@@ -17,21 +17,12 @@
  limitations under the License.
 -->
 
-<!-- This is a "kitchen sink" config file that tests can use.
-     When writting a new test, feel free to add *new* items (plugins,
-     config options, etc...) as long as they don't break any existing
-     tests.  if you need to test something esoteric please add a new
-     "solrconfig-your-esoteric-purpose.xml" config file.
-
-     Note in particular that this test is used by MinimalSchemaTest so
-     Anything added to this file needs to work correctly even if there
-     is now uniqueKey or defaultSearch Field.
-  -->
-
 <config>
 
   <jmx />
 
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
   <!-- Used to specify an alternate directory to hold all index data.
        It defaults to "index" if not present, and should probably
        not be changed if replication is in use. -->

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy1.xml Sun Aug 11 12:19:13 2013
@@ -17,7 +17,6 @@
  limitations under the License.
 -->
 
-
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
@@ -26,11 +25,27 @@
   <requestHandler name="standard" class="solr.StandardRequestHandler"/>
 
   <indexConfig>
-   <deletionPolicy class="solr.SolrDeletionPolicy">
-    <str name="keepOptimizedOnly">true</str>
-    <str name="maxCommitsToKeep">3</str>
-    <str name="maxCommitAge">100MILLISECONDS</str>
-   </deletionPolicy>
+    <!-- we can't include solrconfig.snippet.randomindexconfig.xml because we need
+         to configure an explicit deletion policy, but we still wnat to randomize as much 
+         as possible. 
+    -->
+    <mergePolicy class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <maxIndexingThreads>${solr.tests.maxIndexingThreads}</maxIndexingThreads>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+    
+    <lockType>single</lockType>
+
+
+    <deletionPolicy class="solr.SolrDeletionPolicy">
+      <str name="keepOptimizedOnly">true</str>
+      <str name="maxCommitsToKeep">3</str>
+      <str name="maxCommitAge">100MILLISECONDS</str>
+    </deletionPolicy>
   </indexConfig>
 
 </config>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-delpolicy2.xml Sun Aug 11 12:19:13 2013
@@ -21,6 +21,22 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <indexConfig>
+
+    <!-- we can't include solrconfig.snippet.randomindexconfig.xml because we need
+         to configure an explicit deletion policy, but we still wnat to randomize as much 
+         as possible. 
+    -->
+    <mergePolicy class="${solr.tests.mergePolicy:org.apache.solr.util.RandomMergePolicy}" />
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
+
+    <maxBufferedDocs>${solr.tests.maxBufferedDocs}</maxBufferedDocs>
+    <maxIndexingThreads>${solr.tests.maxIndexingThreads}</maxIndexingThreads>
+    <ramBufferSizeMB>${solr.tests.ramBufferSizeMB}</ramBufferSizeMB>
+    
+    <mergeScheduler class="${solr.tests.mergeScheduler}" />
+    
+    <lockType>single</lockType>
+
     <deletionPolicy class="org.apache.solr.core.FakeDeletionPolicy">
       <str name="var1">value1</str>
 		  <str name="var2">value2</str>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-elevate.xml Sun Aug 11 12:19:13 2013
@@ -32,6 +32,7 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
   <indexConfig>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <lockType>single</lockType>
   </indexConfig>
   
@@ -66,10 +67,8 @@
     </listener>
     -->
 
-
   </updateHandler>
 
-
   <query>
     <!-- Maximum number of clauses in a boolean query... can affect
         range or wildcard queries that expand to big boolean
@@ -77,7 +76,6 @@
     -->
     <maxBooleanClauses>1024</maxBooleanClauses>
 
-
     <!-- Cache specification for Filters or DocSets - unordered set of *all* documents
          that match a particular query.
       -->
@@ -108,19 +106,16 @@
     <!-- set maxSize artificially low to exercise both types of sets -->
     <HashDocSet maxSize="3" loadFactor="0.75"/>
 
-
     <!-- boolToFilterOptimizer converts boolean clauses with zero boost
          into cached filters if the number of docs selected by the clause exceeds
          the threshold (represented as a fraction of the total index)
     -->
     <boolTofilterOptimizer enabled="false" cacheSize="32" threshold=".05"/>
 
-
   </query>
 
   <requestHandler name="/update" class="solr.UpdateRequestHandler"  />
 
-
   <!-- test elevation -->
   <searchComponent name="elevate" class="org.apache.solr.handler.component.QueryElevationComponent" >
     <str name="queryFieldType">string</str>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-enableplugin.xml Sun Aug 11 12:19:13 2013
@@ -28,6 +28,7 @@
 
   <indexConfig>
     <lockType>single</lockType>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
   </indexConfig>
 
   <updateHandler class="solr.DirectUpdateHandler2">
@@ -45,7 +46,6 @@
   <requestHandler name="disabled" class="solr.StandardRequestHandler" enable="false"/>
   <requestHandler name="enabled" class="solr.StandardRequestHandler" enable="true"/>
 
-
   <!-- test query parameter defaults -->
   <requestHandler name="lazy" class="solr.StandardRequestHandler" startup="lazy">
   </requestHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-functionquery.xml Sun Aug 11 12:19:13 2013
@@ -20,6 +20,8 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
  
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
@@ -38,5 +40,4 @@
     <float name="nvlFloatValue">0.0</float>
   </valueSourceParser>
 
-
 </config>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-highlight.xml Sun Aug 11 12:19:13 2013
@@ -20,6 +20,8 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
 
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-indexconfig.xml Sun Aug 11 12:19:13 2013
@@ -22,6 +22,9 @@
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
 
   <indexConfig>
+    <useCompoundFile>${useCompoundFile:false}</useCompoundFile>
     <maxIndexingThreads>123</maxIndexingThreads>
+    <infoStream>true</infoStream>
+    <mergePolicy class="org.apache.solr.util.RandomMergePolicy" />
   </indexConfig>
 </config>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-lazywriter.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-lazywriter.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-lazywriter.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-lazywriter.xml Sun Aug 11 12:19:13 2013
@@ -21,6 +21,7 @@
      DO NOT ADD THINGS TO THIS CONFIG! -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <queryResponseWriter name="velocity" class="solr.VelocityResponseWriter" startup="lazy"/>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml Sun Aug 11 12:19:13 2013
@@ -20,6 +20,8 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
 
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
   <schemaFactory class="ManagedIndexSchemaFactory">
     <bool name="mutable">false</bool>
     <str name="managedSchemaResourceName">managed-schema</str>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master.xml Sun Aug 11 12:19:13 2013
@@ -22,9 +22,7 @@
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <dataDir>${solr.data.dir:}</dataDir>
 
-  <indexConfig>
-    <lockType>single</lockType>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1-keepOneBackup.xml Sun Aug 11 12:19:13 2013
@@ -21,9 +21,7 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
-  <indexConfig>
-    <lockType>single</lockType>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master1.xml Sun Aug 11 12:19:13 2013
@@ -22,9 +22,7 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
-  <indexConfig>
-    <lockType>single</lockType>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master2.xml Sun Aug 11 12:19:13 2013
@@ -22,9 +22,7 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
-  <indexConfig>
-    <lockType>single</lockType>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-master3.xml Sun Aug 11 12:19:13 2013
@@ -22,9 +22,7 @@
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
 
-  <indexConfig>
-    <lockType>single</lockType>
-  </indexConfig>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <updateHandler class="solr.DirectUpdateHandler2">
   </updateHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mergepolicy.xml Sun Aug 11 12:19:13 2013
@@ -26,8 +26,20 @@
     <mergePolicy class="org.apache.lucene.index.TieredMergePolicy">
       <int name="maxMergeAtOnceExplicit">19</int>
       <int name="segmentsPerTier">9</int>
-      <double name="noCFSRatio">1.0</double>
+      <double name="noCFSRatio">0.1</double>
+
+      <!-- Setter for this was moved from the MergePolicies to IndexWriterConfig
+           in Lucene 4.4, so we should treat it the same as a <useCompoundFile>
+           setting and log a warning (instead of failing because the setter is 
+           gone).
+      -->
+      <bool name="useCompoundFile">${useCompoundFile:false}</bool>
+
     </mergePolicy>
+    <mergeScheduler class="org.apache.lucene.index.ConcurrentMergeScheduler">
+      <int name="maxMergeCount">987</int>
+      <int name="maxThreadCount">42</int>
+    </mergeScheduler>
   </indexConfig>
 
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-minimal.xml Sun Aug 11 12:19:13 2013
@@ -28,8 +28,7 @@
   <directoryFactory name="DirectoryFactory"
                     class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
 
-  <indexConfig>
-  </indexConfig>
+  <xi:include href="./solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
 
   <jmx/>
   <updateHandler class="solr.DirectUpdateHandler2">

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mutable-managed-schema.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mutable-managed-schema.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mutable-managed-schema.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-mutable-managed-schema.xml Sun Aug 11 12:19:13 2013
@@ -19,6 +19,8 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
 
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
   <schemaFactory class="ManagedIndexSchemaFactory">
     <bool name="mutable">true</bool>
     <str name="managedSchemaResourceName">managed-schema</str>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-nocache.xml Sun Aug 11 12:19:13 2013
@@ -20,6 +20,7 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
 
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-phrasesuggest.xml Sun Aug 11 12:19:13 2013
@@ -20,6 +20,7 @@
 <!-- solrconfig.xml for a WFST phrase suggester -->
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <dataDir>${solr.data.dir:}</dataDir>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-postingshighlight.xml Sun Aug 11 12:19:13 2013
@@ -21,6 +21,7 @@
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
   <dataDir>${solr.data.dir:}</dataDir>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
   <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
   <requestHandler name="standard" class="solr.StandardRequestHandler">
     <lst name="defaults">

Modified: lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml (original)
+++ lucene/dev/branches/lucene4956/solr/core/src/test-files/solr/collection1/conf/solrconfig-querysender-noquery.xml Sun Aug 11 12:19:13 2013
@@ -19,6 +19,7 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
     <!--  The DirectoryFactory to use for indexes.
         solr.StandardDirectoryFactory, the default, is filesystem based.
         solr.RAMDirectoryFactory is memory based and not persistent. -->
@@ -49,7 +50,6 @@
     <listener event="newSearcher"
               class="org.apache.solr.core.MockEventListener" />
 
-
     <!-- a firstSearcher event is fired whenever a new searcher is being
          prepared but there is no current registered searcher to handle
          requests or to gain prewarming data from. -->