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

svn commit: r1369470 [22/23] - in /lucene/dev/branches/pforcodec_3892: ./ dev-tools/ dev-tools/eclipse/ dev-tools/maven/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/tartarus/snowball/ext/ luce...

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java Sat Aug  4 21:26:10 2012
@@ -282,6 +282,7 @@ public class RecoveryStrategy extends Th
         // last operation at the time of startup had the GAP flag set...
         // this means we were previously doing a full index replication
         // that probably didn't complete and buffering updates in the meantime.
+        log.info("Looks like a previous replication recovery did not complete - skipping peer sync");
         firstTime = false;    // skip peersync
       }
     }
@@ -318,7 +319,7 @@ public class RecoveryStrategy extends Th
             SolrQueryRequest req = new LocalSolrQueryRequest(core,
                 new ModifiableSolrParams());
             core.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
-            log.info("Sync Recovery was successful - registering as Active");
+            log.info("PeerSync Recovery was successful - registering as Active");
             // System.out
             // .println("Sync Recovery was successful - registering as Active "
             // + zkController.getNodeName());
@@ -347,10 +348,10 @@ public class RecoveryStrategy extends Th
             return;
           }
 
-          log.info("Sync Recovery was not successful - trying replication");
+          log.info("PeerSync Recovery was not successful - trying replication");
         }
         //System.out.println("Sync Recovery was not successful - trying replication");
-
+        log.info("Starting Replication Recovery");
         log.info("Begin buffering updates");
         ulog.bufferUpdates();
         replayed = false;
@@ -363,7 +364,7 @@ public class RecoveryStrategy extends Th
           replay(ulog);
           replayed = true;
 
-          log.info("Recovery was successful - registering as Active");
+          log.info("Replication Recovery was successful - registering as Active");
           // if there are pending recovery requests, don't advert as active
           zkController.publish(core.getCoreDescriptor(), ZkStateReader.ACTIVE);
           close = true;

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java Sat Aug  4 21:26:10 2012
@@ -28,7 +28,7 @@ import org.apache.solr.client.solrj.impl
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -132,8 +132,8 @@ public class SyncStrategy {
   
   private boolean areAnyOtherReplicasActive(ZkController zkController,
       ZkNodeProps leaderProps, String collection, String shardId) {
-    CloudState cloudState = zkController.getZkStateReader().getCloudState();
-    Map<String,Slice> slices = cloudState.getSlices(collection);
+    ClusterState clusterState = zkController.getZkStateReader().getClusterState();
+    Map<String,Slice> slices = clusterState.getSlices(collection);
     Slice slice = slices.get(shardId);
     Map<String,ZkNodeProps> shards = slice.getShards();
     for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) {
@@ -142,10 +142,10 @@ public class SyncStrategy {
 //          + state
 //          + shard.getValue().get(ZkStateReader.NODE_NAME_PROP)
 //          + " live: "
-//          + cloudState.liveNodesContain(shard.getValue().get(
+//          + clusterState.liveNodesContain(shard.getValue().get(
 //              ZkStateReader.NODE_NAME_PROP)));
       if ((state.equals(ZkStateReader.ACTIVE))
-          && cloudState.liveNodesContain(shard.getValue().get(
+          && clusterState.liveNodesContain(shard.getValue().get(
               ZkStateReader.NODE_NAME_PROP))
           && !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
               new ZkCoreNodeProps(leaderProps).getCoreUrl())) {

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/cloud/ZkController.java Sat Aug  4 21:26:10 2012
@@ -38,7 +38,7 @@ import org.apache.solr.client.solrj.impl
 import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.OnReconnect;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCmdExecutor;
@@ -121,6 +121,8 @@ public final class ZkController {
   // may accept defaults or use mocks rather than pulling things from a CoreContainer
   private CoreContainer cc;
 
+  protected volatile Overseer overseer;
+
   /**
    * @param cc if null, recovery will not be enabled
    * @param zkServerAddress
@@ -170,10 +172,8 @@ public final class ZkController {
                 shardHandler = cc.getShardHandlerFactory().getShardHandler();
                 adminPath = cc.getAdminPath();
               }
-              
-              ElectionContext context = new OverseerElectionContext(
-                  shardHandler, adminPath,
-                  getNodeName(), zkStateReader);
+              ZkController.this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
+              ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
               overseerElector.joinElection(context);
               zkStateReader.createClusterStateWatchersAndUpdate();
               
@@ -243,6 +243,11 @@ public final class ZkController {
    */
   public void close() {
     try {
+      overseer.close();
+    } catch(Throwable t) {
+      log.error("Error closing overseer", t);
+    }
+    try {
       zkClient.close();
     } catch (InterruptedException e) {
       // Restore the interrupted status
@@ -269,8 +274,8 @@ public final class ZkController {
   /**
    * @return information about the cluster from ZooKeeper
    */
-  public CloudState getCloudState() {
-    return zkStateReader.getCloudState();
+  public ClusterState getClusterState() {
+    return zkStateReader.getClusterState();
   }
 
   /**
@@ -366,8 +371,8 @@ public final class ZkController {
       }
       
       overseerElector = new LeaderElector(zkClient);
-      ElectionContext context = new OverseerElectionContext(shardHandler,
-          adminPath, getNodeName(), zkStateReader);
+      this.overseer = new Overseer(shardHandler, adminPath, zkStateReader);
+      ElectionContext context = new OverseerElectionContext(zkClient, overseer, getNodeName());
       overseerElector.setup(context);
       overseerElector.joinElection(context);
       zkStateReader.createClusterStateWatchersAndUpdate();
@@ -540,18 +545,18 @@ public final class ZkController {
     String leaderUrl = getLeaderProps(collection, cloudDesc.getShardId()).getCoreUrl();
     
     // now wait until our currently cloud state contains the latest leader
-    String cloudStateLeader = zkStateReader.getLeaderUrl(collection, cloudDesc.getShardId(), 30000);
+    String clusterStateLeader = zkStateReader.getLeaderUrl(collection, shardId, 30000);
     int tries = 0;
-    while (!leaderUrl.equals(cloudStateLeader)) {
+    while (!leaderUrl.equals(clusterStateLeader)) {
       if (tries == 60) {
         throw new SolrException(ErrorCode.SERVER_ERROR,
             "There is conflicting information about the leader of shard: "
-                + cloudDesc.getShardId());
+                + cloudDesc.getShardId() + " our state says:" + clusterStateLeader + " but zookeeper says:" + leaderUrl);
       }
       Thread.sleep(1000);
       tries++;
-      cloudStateLeader = zkStateReader.getLeaderUrl(collection,
-          cloudDesc.getShardId(), 30000);
+      clusterStateLeader = zkStateReader.getLeaderUrl(collection, shardId, 30000);
+      leaderUrl = getLeaderProps(collection, cloudDesc.getShardId()).getCoreUrl();
     }
     
     String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
@@ -603,7 +608,7 @@ public final class ZkController {
     }
     
     // make sure we have an update cluster state right away
-    zkStateReader.updateCloudState(true);
+    zkStateReader.updateClusterState(true);
 
     return shardId;
   }
@@ -617,10 +622,10 @@ public final class ZkController {
    * @throws KeeperException
    * @throws InterruptedException
    */
-  private ZkCoreNodeProps getLeaderProps(final String collection, final String slice)
-      throws KeeperException, InterruptedException {
+  private ZkCoreNodeProps getLeaderProps(final String collection,
+      final String slice) throws KeeperException, InterruptedException {
     int iterCount = 60;
-    while (iterCount-- > 0)
+    while (iterCount-- > 0) {
       try {
         byte[] data = zkClient.getData(
             ZkStateReader.getShardLeadersPath(collection, slice), null, null,
@@ -631,6 +636,7 @@ public final class ZkController {
       } catch (NoNodeException e) {
         Thread.sleep(500);
       }
+    }
     throw new RuntimeException("Could not get leader props");
   }
 
@@ -735,7 +741,7 @@ public final class ZkController {
   }
 
   private boolean needsToBeAssignedShardId(final CoreDescriptor desc,
-      final CloudState state, final String shardZkNodeName) {
+      final ClusterState state, final String shardZkNodeName) {
 
     final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
     
@@ -935,7 +941,7 @@ public final class ZkController {
     final String shardZkNodeName = getNodeName() + "_" + coreName;
     int retryCount = 120;
     while (retryCount-- > 0) {
-      final String shardId = zkStateReader.getCloudState().getShardId(
+      final String shardId = zkStateReader.getClusterState().getShardId(
           shardZkNodeName);
       if (shardId != null) {
         return shardId;
@@ -1003,7 +1009,7 @@ public final class ZkController {
     // this also gets us our assigned shard id if it was not specified
     publish(cd, ZkStateReader.DOWN); 
     String shardZkNodeName = getCoreNodeName(cd);
-    if (cd.getCloudDescriptor().getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getCloudState(), shardZkNodeName)) {
+    if (cd.getCloudDescriptor().getShardId() == null && needsToBeAssignedShardId(cd, zkStateReader.getClusterState(), shardZkNodeName)) {
       String shardId;
       shardId = doGetShardIdProcess(cd.getName(), cd.getCloudDescriptor());
       cd.getCloudDescriptor().setShardId(shardId);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java Sat Aug  4 21:26:10 2012
@@ -26,7 +26,7 @@ import org.apache.solr.client.solrj.requ
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.OverseerCollectionProcessor;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -141,9 +141,9 @@ public class CollectionsHandler extends 
     String collection = req.getParams().required().get("collection");
     String shard = req.getParams().required().get("shard");
     
-    CloudState cloudState = coreContainer.getZkController().getCloudState();
+    ClusterState clusterState = coreContainer.getZkController().getClusterState();
     
-    ZkNodeProps leaderProps = cloudState.getLeader(collection, shard);
+    ZkNodeProps leaderProps = clusterState.getLeader(collection, shard);
     ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(leaderProps);
     
     HttpSolrServer server = new HttpSolrServer(nodeProps.getBaseUrl());

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Sat Aug  4 21:26:10 2012
@@ -35,7 +35,7 @@ import org.apache.solr.cloud.SyncStrateg
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -768,16 +768,16 @@ public class CoreAdminHandler extends Re
           // to accept updates
           CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
               .getCloudDescriptor();
-          CloudState cloudState = coreContainer.getZkController()
-              .getCloudState();
+          ClusterState clusterState = coreContainer.getZkController()
+              .getClusterState();
           String collection = cloudDescriptor.getCollectionName();
-          Slice slice = cloudState.getSlice(collection,
+          Slice slice = clusterState.getSlice(collection,
               cloudDescriptor.getShardId());
           if (slice != null) {
             ZkNodeProps nodeProps = slice.getShards().get(coreNodeName);
             if (nodeProps != null) {
               state = nodeProps.get(ZkStateReader.STATE_PROP);
-              live = cloudState.liveNodesContain(nodeName);
+              live = clusterState.liveNodesContain(nodeName);
               if (nodeProps != null && state.equals(waitForState)) {
                 if (checkLive == null) {
                   break;

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Sat Aug  4 21:26:10 2012
@@ -41,7 +41,7 @@ import org.apache.solr.cloud.CloudDescri
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -255,7 +255,7 @@ public class HttpShardHandler extends Sh
     if (rb.isDistrib) {
       // since the cost of grabbing cloud state is still up in the air, we grab it only
       // if we need it.
-      CloudState cloudState = null;
+      ClusterState clusterState = null;
       Map<String,Slice> slices = null;
       CoreDescriptor coreDescriptor = req.getCore().getCoreDescriptor();
       CloudDescriptor cloudDescriptor = coreDescriptor.getCloudDescriptor();
@@ -280,7 +280,7 @@ public class HttpShardHandler extends Sh
       } else if (zkController != null) {
         // we weren't provided with a list of slices to query, so find the list that will cover the complete index
 
-        cloudState =  zkController.getCloudState();
+        clusterState =  zkController.getClusterState();
 
         // This can be more efficient... we only record the name, even though we
         // have the shard info we need in the next step of mapping slice->shards
@@ -301,12 +301,12 @@ public class HttpShardHandler extends Sh
           // cloud state and add them to the Map 'slices'.
           for (int i = 0; i < collectionList.size(); i++) {
             String collection = collectionList.get(i);
-            ClientUtils.appendMap(collection, slices, cloudState.getSlices(collection));
+            ClientUtils.appendMap(collection, slices, clusterState.getSlices(collection));
           }
         } else {
           // If no collections were specified, default to the collection for
           // this core.
-          slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
+          slices = clusterState.getSlices(cloudDescriptor.getCollectionName());
           if (slices == null) {
             throw new SolrException(ErrorCode.BAD_REQUEST,
                 "Could not find collection:"
@@ -334,9 +334,9 @@ public class HttpShardHandler extends Sh
       if (zkController != null) {
         for (int i=0; i<rb.shards.length; i++) {
           if (rb.shards[i] == null) {
-            if (cloudState == null) {
-              cloudState =  zkController.getCloudState();
-              slices = cloudState.getSlices(cloudDescriptor.getCollectionName());
+            if (clusterState == null) {
+              clusterState =  zkController.getClusterState();
+              slices = clusterState.getSlices(cloudDescriptor.getCollectionName());
             }
             String sliceName = rb.slices[i];
 
@@ -353,7 +353,7 @@ public class HttpShardHandler extends Sh
             Map<String, ZkNodeProps> sliceShards = slice.getShards();
 
             // For now, recreate the | delimited list of equivalent servers
-            Set<String> liveNodes = cloudState.getLiveNodes();
+            Set<String> liveNodes = clusterState.getLiveNodes();
             StringBuilder sliceShardsStr = new StringBuilder();
             boolean first = true;
             for (ZkNodeProps nodeProps : sliceShards.values()) {

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java Sat Aug  4 21:26:10 2012
@@ -352,14 +352,14 @@ public class RealTimeGetComponent extend
 
       String collection = cloudDescriptor.getCollectionName();
 
-      CloudState cloudState = zkController.getCloudState();
+      ClusterState clusterState = zkController.getClusterState();
       
       Map<String, List<String>> shardToId = new HashMap<String, List<String>>();
       for (String id : allIds) {
         BytesRef br = new BytesRef();
         sf.getType().readableToIndexed(id, br);
         int hash = Hash.murmurhash3_x86_32(br.bytes, br.offset, br.length, 0);
-        String shard = cloudState.getShard(hash,  collection);
+        String shard = clusterState.getShard(hash,  collection);
 
         List<String> idsForShard = shardToId.get(shard);
         if (idsForShard == null) {

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

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/schema/LatLonType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/schema/LatLonType.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/schema/LatLonType.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/schema/LatLonType.java Sat Aug  4 21:26:10 2012
@@ -74,18 +74,18 @@ public class LatLonType extends Abstract
       }
       //latitude
       SchemaField lat = subField(field, i);
-      f[i] = lat.createField(String.valueOf(latLon[LAT]), lat.omitNorms() ? 1F : boost);
+      f[i] = lat.createField(String.valueOf(latLon[LAT]), lat.indexed() && !lat.omitNorms() ? boost : 1f);
       i++;
       //longitude
       SchemaField lon = subField(field, i);
-      f[i] = lon.createField(String.valueOf(latLon[LON]), lon.omitNorms() ? 1F : boost);
+      f[i] = lon.createField(String.valueOf(latLon[LON]), lon.indexed() && !lon.omitNorms() ? boost : 1f);
 
     }
 
     if (field.stored()) {
       FieldType customType = new FieldType();
       customType.setStored(true);
-      f[f.length - 1] = createField(field.getName(), externalVal, customType, boost);
+      f[f.length - 1] = createField(field.getName(), externalVal, customType, 1f);
     }
     return f;
   }

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

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

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

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/SolrCmdDistributor.java Sat Aug  4 21:26:10 2012
@@ -24,16 +24,12 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.Future;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
@@ -48,6 +44,7 @@ import org.apache.solr.common.cloud.ZkCo
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.AdjustableSemaphore;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -58,15 +55,17 @@ public class SolrCmdDistributor {
   public static Logger log = LoggerFactory.getLogger(SolrCmdDistributor.class);
   
   // TODO: shut this thing down
-  // TODO: this cannot be per instance...
-  static BoundedExecutor commExecutor;
+  static ThreadPoolExecutor commExecutor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 5,
+      TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+      new DefaultSolrThreadFactory("cmdDistribExecutor"));;
 
   static final HttpClient client;
+  static AdjustableSemaphore semaphore = new AdjustableSemaphore(8);
   
   static {
     ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS, 200);
-    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, 8);
+    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS, 500);
+    params.set(HttpClientUtil.PROP_MAX_CONNECTIONS_PER_HOST, 16);
     client = HttpClientUtil.createClient(params);
   }
   
@@ -92,20 +91,14 @@ public class SolrCmdDistributor {
   }
   
   public SolrCmdDistributor(int numHosts) {
-
-    BoundedExecutor executor = null;
-    synchronized (SolrCmdDistributor.class) {
-      if (commExecutor == null || commExecutor.getMaximumPoolSize() != numHosts) {
-        // we don't shutdown the previous because all it's threads will die
-        int maxPoolSize = Math.max(8, (numHosts-1) * 8);
-        commExecutor = new BoundedExecutor(0, maxPoolSize, 5,
-            TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(maxPoolSize * 2),
-            new DefaultSolrThreadFactory("cmdDistribExecutor"));
-      }
-      executor = commExecutor;
-    }
+    int maxPermits = Math.max(8, (numHosts - 1) * 8);
     
-    completionService = new ExecutorCompletionService<Request>(executor);
+    // limits how many tasks can actually execute at once
+    if (maxPermits != semaphore.getMaxPermits()) {
+      semaphore.setMaxPermits(maxPermits);
+    }
+
+    completionService = new ExecutorCompletionService<Request>(commExecutor);
     pending = new HashSet<Future<Request>>();
   }
   
@@ -221,7 +214,7 @@ public class SolrCmdDistributor {
  
     for (Node node : nodes) {
       List<AddRequest> alist = adds.get(node);
-      if (alist == null || alist.size() < limit) return false;
+      if (alist == null || alist.size() < limit) continue;
   
       UpdateRequestExt ureq = new UpdateRequestExt();
       
@@ -256,7 +249,7 @@ public class SolrCmdDistributor {
     Set<Node> nodes = deletes.keySet();
     for (Node node : nodes) {
       List<DeleteRequest> dlist = deletes.get(node);
-      if (dlist == null || dlist.size() < limit) return false;
+      if (dlist == null || dlist.size() < limit) continue;
       UpdateRequestExt ureq = new UpdateRequestExt();
       
       ModifiableSolrParams combinedParams = new ModifiableSolrParams();
@@ -343,11 +336,17 @@ public class SolrCmdDistributor {
           } else {
             clonedRequest.rspCode = -1;
           }
+        } finally {
+          semaphore.release();
         }
         return clonedRequest;
       }
     };
-    
+    try {
+      semaphore.acquire();
+    } catch (InterruptedException e) {
+      throw new RuntimeException();
+    }
     pending.add(completionService.submit(task));
     
   }
@@ -514,39 +513,6 @@ public class SolrCmdDistributor {
     }
   }
   
-  public class BoundedExecutor extends ThreadPoolExecutor {
-    private final Semaphore semaphore;
-    
-    public BoundedExecutor(int corePoolSize,
-        int maximumPoolSize, long keepAliveTime, TimeUnit unit,
-        BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory) {
-      super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory);
-      this.semaphore = new Semaphore(maximumPoolSize);
-    }
-
-    @Override
-    public void execute(final Runnable command) {
-      try {
-        semaphore.acquire();
-      } catch (InterruptedException e1) {
-        throw new RuntimeException();
-      }
-      try {
-        super.execute(new Runnable() {
-          public void run() {
-            try {
-              command.run();
-            } finally {
-              semaphore.release();
-            }
-          }
-        });
-      } catch (RejectedExecutionException e) {
-        semaphore.release();
-        throw e;
-      }
-    }
-}
 }
 
 

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Sat Aug  4 21:26:10 2012
@@ -36,7 +36,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -158,17 +158,20 @@ public class DistributedUpdateProcessor 
     CoreDescriptor coreDesc = req.getCore().getCoreDescriptor();
     
     this.zkEnabled  = coreDesc.getCoreContainer().isZooKeeperAware();
+    zkController = req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
+    if (zkEnabled) {
+      numNodes =  zkController.getZkStateReader().getClusterState().getLiveNodes().size();
+    }
     //this.rsp = reqInfo != null ? reqInfo.getRsp() : null;
 
-    
-    zkController = req.getCore().getCoreDescriptor().getCoreContainer().getZkController();
+   
     
     cloudDesc = coreDesc.getCloudDescriptor();
     
     if (cloudDesc != null) {
       collection = cloudDesc.getCollectionName();
     }
-    
+
     cmdDistrib = new SolrCmdDistributor(numNodes);
   }
 
@@ -178,13 +181,13 @@ public class DistributedUpdateProcessor 
     // if we are in zk mode...
     if (zkEnabled) {
       // set num nodes
-      numNodes = zkController.getCloudState().getLiveNodes().size();
+      numNodes = zkController.getClusterState().getLiveNodes().size();
       
       // the leader is...
       // TODO: if there is no leader, wait and look again
       // TODO: we are reading the leader from zk every time - we should cache
       // this and watch for changes?? Just pull it from ZkController cluster state probably?
-      String shardId = getShard(hash, collection, zkController.getCloudState()); // get the right shard based on the hash...
+      String shardId = getShard(hash, collection, zkController.getClusterState()); // get the right shard based on the hash...
 
       try {
         // TODO: if we find out we cannot talk to zk anymore, we should probably realize we are not
@@ -249,11 +252,11 @@ public class DistributedUpdateProcessor 
   }
 
 
-  private String getShard(int hash, String collection, CloudState cloudState) {
+  private String getShard(int hash, String collection, ClusterState clusterState) {
     // ranges should be part of the cloud state and eventually gotten from zk
 
     // get the shard names
-    return cloudState.getShard(hash, collection);
+    return clusterState.getShard(hash, collection);
   }
 
   // used for deleteByQuery to get the list of nodes this leader should forward to
@@ -695,11 +698,11 @@ public class DistributedUpdateProcessor 
     if (zkEnabled && DistribPhase.NONE == phase) {
       boolean leaderForAnyShard = false;  // start off by assuming we are not a leader for any shard
 
-      Map<String,Slice> slices = zkController.getCloudState().getSlices(collection);
+      Map<String,Slice> slices = zkController.getClusterState().getSlices(collection);
       if (slices == null) {
         throw new SolrException(ErrorCode.BAD_REQUEST,
             "Cannot find collection:" + collection + " in "
-                + zkController.getCloudState().getCollections());
+                + zkController.getClusterState().getCollections());
       }
 
       ModifiableSolrParams params = new ModifiableSolrParams(req.getParams());
@@ -994,13 +997,13 @@ public class DistributedUpdateProcessor 
 
   
   private List<Node> getCollectionUrls(SolrQueryRequest req, String collection, String shardZkNodeName) {
-    CloudState cloudState = req.getCore().getCoreDescriptor()
-        .getCoreContainer().getZkController().getCloudState();
+    ClusterState clusterState = req.getCore().getCoreDescriptor()
+        .getCoreContainer().getZkController().getClusterState();
     List<Node> urls = new ArrayList<Node>();
-    Map<String,Slice> slices = cloudState.getSlices(collection);
+    Map<String,Slice> slices = clusterState.getSlices(collection);
     if (slices == null) {
       throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
-          "Could not find collection in zk: " + cloudState);
+          "Could not find collection in zk: " + clusterState);
     }
     for (Map.Entry<String,Slice> sliceEntry : slices.entrySet()) {
       Slice replicas = slices.get(sliceEntry.getKey());
@@ -1009,7 +1012,7 @@ public class DistributedUpdateProcessor 
       
       for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) {
         ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
-        if (cloudState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) {
+        if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) {
           urls.add(new StdNode(nodeProps));
         }
       }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test-files/solr/collection1/conf/solrconfig-tlog.xml Sat Aug  4 21:26:10 2012
@@ -39,4 +39,6 @@
     </updateLog>
   </updateHandler>
 
+  <requestHandler name="/admin/" class="org.apache.solr.handler.admin.AdminHandlers" />
+
 </config>

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/AbstractDistributedZkTestCase.java Sat Aug  4 21:26:10 2012
@@ -24,7 +24,7 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -125,21 +125,21 @@ public abstract class AbstractDistribute
     while (cont) {
       if (verbose) System.out.println("-");
       boolean sawLiveRecovering = false;
-      zkStateReader.updateCloudState(true);
-      CloudState cloudState = zkStateReader.getCloudState();
-      Map<String,Slice> slices = cloudState.getSlices(collection);
+      zkStateReader.updateClusterState(true);
+      ClusterState clusterState = zkStateReader.getClusterState();
+      Map<String,Slice> slices = clusterState.getSlices(collection);
       for (Map.Entry<String,Slice> entry : slices.entrySet()) {
         Map<String,ZkNodeProps> shards = entry.getValue().getShards();
         for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) {
           if (verbose) System.out.println("rstate:"
               + shard.getValue().get(ZkStateReader.STATE_PROP)
               + " live:"
-              + cloudState.liveNodesContain(shard.getValue().get(
+              + clusterState.liveNodesContain(shard.getValue().get(
                   ZkStateReader.NODE_NAME_PROP)));
           String state = shard.getValue().get(ZkStateReader.STATE_PROP);
           if ((state.equals(ZkStateReader.RECOVERING) || state
               .equals(ZkStateReader.SYNC) || state.equals(ZkStateReader.DOWN))
-              && cloudState.liveNodesContain(shard.getValue().get(
+              && clusterState.liveNodesContain(shard.getValue().get(
                   ZkStateReader.NODE_NAME_PROP))) {
             sawLiveRecovering = true;
           }
@@ -168,9 +168,9 @@ public abstract class AbstractDistribute
   protected void assertAllActive(String collection,ZkStateReader zkStateReader)
       throws KeeperException, InterruptedException {
 
-      zkStateReader.updateCloudState(true);
-      CloudState cloudState = zkStateReader.getCloudState();
-      Map<String,Slice> slices = cloudState.getSlices(collection);
+      zkStateReader.updateClusterState(true);
+      ClusterState clusterState = zkStateReader.getClusterState();
+      Map<String,Slice> slices = clusterState.getSlices(collection);
       if (slices == null) {
         throw new IllegalArgumentException("Cannot find collection:" + collection);
       }

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java Sat Aug  4 21:26:10 2012
@@ -57,7 +57,7 @@ import org.apache.solr.common.SolrDocume
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -355,7 +355,7 @@ public class BasicDistributedZkTest exte
       HttpSolrServer collectionClient = new HttpSolrServer(url);
       
       // poll for a second - it can take a moment before we are ready to serve
-      waitForNon404(collectionClient);
+      waitForNon404or503(collectionClient);
     }
     
     List<String> collectionNameList = new ArrayList<String>();
@@ -451,7 +451,7 @@ public class BasicDistributedZkTest exte
   private void collectStartTimes(String collectionName,
       Map<String,Long> urlToTime) throws SolrServerException, IOException {
     Map<String,Map<String,Slice>> collections = solrj.getZkStateReader()
-        .getCloudState().getCollectionStates();
+        .getClusterState().getCollectionStates();
     if (collections.containsKey(collectionName)) {
       Map<String,Slice> slices = collections.get(collectionName);
 
@@ -478,8 +478,8 @@ public class BasicDistributedZkTest exte
   }
 
   private String getUrlFromZk(String collection) {
-    CloudState cloudState = solrj.getZkStateReader().getCloudState();
-    Map<String,Slice> slices = cloudState.getCollectionStates().get(collection);
+    ClusterState clusterState = solrj.getZkStateReader().getClusterState();
+    Map<String,Slice> slices = clusterState.getCollectionStates().get(collection);
     
     if (slices == null) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "Could not find collection:" + collection);
@@ -491,7 +491,7 @@ public class BasicDistributedZkTest exte
       Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet();
       for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) {
         final ZkNodeProps node = shardEntry.getValue();
-        if (cloudState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
+        if (clusterState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
           return new ZkCoreNodeProps(node).getCoreUrl();
         }
       }
@@ -500,20 +500,21 @@ public class BasicDistributedZkTest exte
     throw new RuntimeException("Could not find a live node for collection:" + collection);
   }
 
-  private void waitForNon404(HttpSolrServer collectionClient)
+  private void waitForNon404or503(HttpSolrServer collectionClient)
       throws Exception {
-    
+    SolrException exp = null;
     long timeoutAt = System.currentTimeMillis() + 30000;
     
     while (System.currentTimeMillis() < timeoutAt) {
       boolean missing = false;
+
       try {
         collectionClient.query(new SolrQuery("*:*"));
       } catch (SolrException e) {
-        // How do I get the response code!?
-        if (!e.getMessage().contains("(404)")) {
+        if (!(e.code() == 403 || e.code() == 503)) {
           throw e;
         }
+        exp = e;
         missing = true;
       }
       if (!missing) {
@@ -522,7 +523,7 @@ public class BasicDistributedZkTest exte
       Thread.sleep(50);
     }
     printLayout();
-    fail("Could not find the new collection - 404 : " + collectionClient.getBaseURL());
+    fail("Could not find the new collection - " + exp.code() + " : " + collectionClient.getBaseURL());
   }
 
   private void checkForCollection(String collectionName, int expectedSlices)
@@ -532,9 +533,9 @@ public class BasicDistributedZkTest exte
     boolean found = false;
     boolean sliceMatch = false;
     while (System.currentTimeMillis() < timeoutAt) {
-      solrj.getZkStateReader().updateCloudState(true);
-      CloudState cloudState = solrj.getZkStateReader().getCloudState();
-      Map<String,Map<String,Slice>> collections = cloudState
+      solrj.getZkStateReader().updateClusterState(true);
+      ClusterState clusterState = solrj.getZkStateReader().getClusterState();
+      Map<String,Map<String,Slice>> collections = clusterState
           .getCollectionStates();
       if (collections.containsKey(collectionName)) {
         Map<String,Slice> slices = collections.get(collectionName);
@@ -580,9 +581,9 @@ public class BasicDistributedZkTest exte
     long timeoutAt = System.currentTimeMillis() + 15000;
     boolean found = true;
     while (System.currentTimeMillis() < timeoutAt) {
-      solrj.getZkStateReader().updateCloudState(true);
-      CloudState cloudState = solrj.getZkStateReader().getCloudState();
-      Map<String,Map<String,Slice>> collections = cloudState
+      solrj.getZkStateReader().updateClusterState(true);
+      ClusterState clusterState = solrj.getZkStateReader().getClusterState();
+      Map<String,Map<String,Slice>> collections = clusterState
           .getCollectionStates();
       if (!collections.containsKey(collectionName)) {
         found = false;
@@ -772,8 +773,8 @@ public class BasicDistributedZkTest exte
     
     // we added a role of none on these creates - check for it
     ZkStateReader zkStateReader = solrj.getZkStateReader();
-    zkStateReader.updateCloudState(true);
-    Map<String,Slice> slices = zkStateReader.getCloudState().getSlices(oneInstanceCollection2);
+    zkStateReader.updateClusterState(true);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getSlices(oneInstanceCollection2);
     assertNotNull(slices);
     String roles = slices.get("slice1").getShards().values().iterator().next().get(ZkStateReader.ROLES_PROP);
     assertEquals("none", roles);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/BasicZkTest.java Sat Aug  4 21:26:10 2012
@@ -126,8 +126,8 @@ public class BasicZkTest extends Abstrac
     
     // ensure zk still thinks node is up
     assertTrue(
-        zkController.getCloudState().getLiveNodes().toString(),
-        zkController.getCloudState().liveNodesContain(
+        zkController.getClusterState().getLiveNodes().toString(),
+        zkController.getClusterState().liveNodesContain(
             zkController.getNodeName()));
 
     // test maxint

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkey.java Sat Aug  4 21:26:10 2012
@@ -232,7 +232,7 @@ public class ChaosMonkey {
   }
 
   private String getRandomSlice() {
-    Map<String,Slice> slices = zkStateReader.getCloudState().getSlices(collection);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getSlices(collection);
     
     List<String> sliceKeyList = new ArrayList<String>(slices.size());
     sliceKeyList.addAll(slices.keySet());
@@ -259,9 +259,9 @@ public class ChaosMonkey {
       boolean running = true;
       
       // get latest cloud state
-      zkStateReader.updateCloudState(true);
+      zkStateReader.updateClusterState(true);
       
-      Slice theShards = zkStateReader.getCloudState().getSlices(collection)
+      Slice theShards = zkStateReader.getClusterState().getSlices(collection)
           .get(slice);
       
       ZkNodeProps props = theShards.getShards().get(cloudJetty.coreNodeName);
@@ -275,19 +275,19 @@ public class ChaosMonkey {
       
       if (!cloudJetty.jetty.isRunning()
           || !state.equals(ZkStateReader.ACTIVE)
-          || !zkStateReader.getCloudState().liveNodesContain(nodeName)) {
+          || !zkStateReader.getClusterState().liveNodesContain(nodeName)) {
         running = false;
       }
       
       if (cloudJetty.jetty.isRunning()
           && state.equals(ZkStateReader.RECOVERING)
-          && zkStateReader.getCloudState().liveNodesContain(nodeName)) {
+          && zkStateReader.getClusterState().liveNodesContain(nodeName)) {
         numRecovering++;
       }
       
       if (cloudJetty.jetty.isRunning()
           && state.equals(ZkStateReader.ACTIVE)
-          && zkStateReader.getCloudState().liveNodesContain(nodeName)) {
+          && zkStateReader.getClusterState().liveNodesContain(nodeName)) {
         numActive++;
       }
       
@@ -336,7 +336,7 @@ public class ChaosMonkey {
   
   public SolrServer getRandomClient(String slice) throws KeeperException, InterruptedException {
     // get latest cloud state
-    zkStateReader.updateCloudState(true);
+    zkStateReader.updateClusterState(true);
 
     // get random shard
     List<SolrServer> clients = shardToClient.get(slice);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java Sat Aug  4 21:26:10 2012
@@ -148,8 +148,8 @@ public class ChaosMonkeyNothingIsSafeTes
       
       // TODO: assert we didnt kill everyone
       
-      zkStateReader.updateCloudState(true);
-      assertTrue(zkStateReader.getCloudState().getLiveNodes().size() > 0);
+      zkStateReader.updateClusterState(true);
+      assertTrue(zkStateReader.getClusterState().getLiveNodes().size() > 0);
       
       checkShardConsistency(false, true);
       

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudTest.java Sat Aug  4 21:26:10 2012
@@ -44,7 +44,7 @@ import org.apache.solr.common.SolrDocume
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -214,11 +214,11 @@ public class FullSolrCloudTest extends A
     }
     
     // wait until shards have started registering...
-    while (!zkStateReader.getCloudState().getCollections()
+    while (!zkStateReader.getClusterState().getCollections()
         .contains(DEFAULT_COLLECTION)) {
       Thread.sleep(500);
     }
-    while (zkStateReader.getCloudState().getSlices(DEFAULT_COLLECTION).size() != sliceCount) {
+    while (zkStateReader.getClusterState().getSlices(DEFAULT_COLLECTION).size() != sliceCount) {
       Thread.sleep(500);
     }
     
@@ -331,7 +331,7 @@ public class FullSolrCloudTest extends A
   }
 
   private int getNumShards(String defaultCollection) {
-    Map<String,Slice> slices = this.zkStateReader.getCloudState().getSlices(defaultCollection);
+    Map<String,Slice> slices = this.zkStateReader.getClusterState().getSlices(defaultCollection);
     int cnt = 0;
     for (Map.Entry<String,Slice> entry : slices.entrySet()) {
       cnt += entry.getValue().getShards().size();
@@ -354,16 +354,16 @@ public class FullSolrCloudTest extends A
   
   protected void updateMappingsFromZk(List<JettySolrRunner> jettys,
       List<SolrServer> clients) throws Exception {
-    zkStateReader.updateCloudState(true);
+    zkStateReader.updateClusterState(true);
     cloudJettys.clear();
     shardToJetty.clear();
     
-    CloudState cloudState = zkStateReader.getCloudState();
-    Map<String,Slice> slices = cloudState.getSlices(DEFAULT_COLLECTION);
+    ClusterState clusterState = zkStateReader.getClusterState();
+    Map<String,Slice> slices = clusterState.getSlices(DEFAULT_COLLECTION);
     
     if (slices == null) {
       throw new RuntimeException("No slices found for collection "
-          + DEFAULT_COLLECTION + " in " + cloudState.getCollections());
+          + DEFAULT_COLLECTION + " in " + clusterState.getCollections());
     }
     
     List<CloudSolrServerClient> theClients = new ArrayList<CloudSolrServerClient>();
@@ -1053,7 +1053,7 @@ public class FullSolrCloudTest extends A
     assertEquals(
         "The client count does not match up with the shard count for slice:"
             + shard,
-        zkStateReader.getCloudState().getSlice(DEFAULT_COLLECTION, shard)
+        zkStateReader.getClusterState().getSlice(DEFAULT_COLLECTION, shard)
             .getShards().size(), solrJetties.size());
 
     SolrServer lastClient = null;
@@ -1078,7 +1078,7 @@ public class FullSolrCloudTest extends A
       
       boolean live = false;
       String nodeName = props.get(ZkStateReader.NODE_NAME_PROP);
-      if (zkStateReader.getCloudState().liveNodesContain(nodeName)) {
+      if (zkStateReader.getClusterState().liveNodesContain(nodeName)) {
         live = true;
       }
       if (verbose) System.err.println(" live:" + live);
@@ -1233,18 +1233,18 @@ public class FullSolrCloudTest extends A
     ZkStateReader zk = new ZkStateReader(zkServer.getZkAddress(), 10000,
         AbstractZkTestCase.TIMEOUT);
     Map<String,Slice> slices = null;
-    CloudState cloudState;
+    ClusterState clusterState;
     try {
       zk.createClusterStateWatchersAndUpdate();
-      cloudState = zk.getCloudState();
-      slices = cloudState.getSlices(DEFAULT_COLLECTION);
+      clusterState = zk.getClusterState();
+      slices = clusterState.getSlices(DEFAULT_COLLECTION);
     } finally {
       zk.close();
     }
     
     if (slices == null) {
       throw new RuntimeException("Could not find collection "
-          + DEFAULT_COLLECTION + " in " + cloudState.getCollections());
+          + DEFAULT_COLLECTION + " in " + clusterState.getCollections());
     }
     
     for (CloudJettyRunner cjetty : cloudJettys) {
@@ -1266,7 +1266,7 @@ public class FullSolrCloudTest extends A
       String currentState = cjetty.info.get(ZkStateReader.STATE_PROP);
       if (currentState != null
           && currentState.equals(ZkStateReader.ACTIVE)
-          && zkStateReader.getCloudState().liveNodesContain(
+          && zkStateReader.getClusterState().liveNodesContain(
               cjetty.info.get(ZkStateReader.NODE_NAME_PROP))) {
         SolrQuery query = new SolrQuery("*:*");
         query.set("distrib", false);
@@ -1509,7 +1509,7 @@ public class FullSolrCloudTest extends A
   protected void waitToSeeNotLive(ZkStateReader zkStateReader,
       CloudJettyRunner cjetty) throws InterruptedException {
     int tries = 0;
-    while (zkStateReader.getCloudState()
+    while (zkStateReader.getClusterState()
         .liveNodesContain(cjetty.info.get(ZkStateReader.NODE_NAME_PROP))) {
       if (tries++ == 120) {
         fail("Shard still reported as live in zk");

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java Sat Aug  4 21:26:10 2012
@@ -36,7 +36,7 @@ import javax.xml.parsers.ParserConfigura
 
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -153,7 +153,7 @@ public class OverseerTest extends SolrTe
     }
     
     private String getShardId(final String coreName) {
-      Map<String,Slice> slices = zkStateReader.getCloudState().getSlices(
+      Map<String,Slice> slices = zkStateReader.getClusterState().getSlices(
           collection);
       if (slices != null) {
         for (Slice slice : slices.values()) {
@@ -340,8 +340,8 @@ public class OverseerTest extends SolrTe
       
       // make sure all cores have been assigned a id in cloudstate
       for (int i = 0; i < 40; i++) {
-        reader.updateCloudState(true);
-        CloudState state = reader.getCloudState();
+        reader.updateClusterState(true);
+        ClusterState state = reader.getClusterState();
         Map<String,Slice> slices = state.getSlices("collection1");
         int count = 0;
         for (String name : slices.keySet()) {
@@ -419,8 +419,8 @@ public class OverseerTest extends SolrTe
   private void waitForCollections(ZkStateReader stateReader, String... collections) throws InterruptedException, KeeperException {
     int maxIterations = 100;
     while (0 < maxIterations--) {
-      stateReader.updateCloudState(true);
-      final CloudState state = stateReader.getCloudState();
+      stateReader.updateClusterState(true);
+      final ClusterState state = stateReader.getClusterState();
       Set<String> availableCollections = state.getCollections();
       int availableCount = 0;
       for(String requiredCollection: collections) {
@@ -431,7 +431,7 @@ public class OverseerTest extends SolrTe
         Thread.sleep(50);
       }
     }
-    log.warn("Timeout waiting for collections: " + Arrays.asList(collections) + " state:" + stateReader.getCloudState());
+    log.warn("Timeout waiting for collections: " + Arrays.asList(collections) + " state:" + stateReader.getClusterState());
   }
   
   @Test
@@ -472,8 +472,8 @@ public class OverseerTest extends SolrTe
       
       waitForCollections(reader, "collection1");
 
-      assertEquals(reader.getCloudState().toString(), ZkStateReader.RECOVERING,
-          reader.getCloudState().getSlice("collection1", "shard1").getShards()
+      assertEquals(reader.getClusterState().toString(), ZkStateReader.RECOVERING,
+          reader.getClusterState().getSlice("collection1", "shard1").getShards()
               .get("node1_core1").get(ZkStateReader.STATE_PROP));
 
       //publish node state (active)
@@ -503,7 +503,7 @@ public class OverseerTest extends SolrTe
     int maxIterations = 100;
     String coreState = null;
     while(maxIterations-->0) {
-      Slice slice = reader.getCloudState().getSlice("collection1", "shard1");
+      Slice slice = reader.getClusterState().getSlice("collection1", "shard1");
       if(slice!=null) {
         coreState = slice.getShards().get("node1_core1").get(ZkStateReader.STATE_PROP);
         if(coreState.equals(expectedState)) {
@@ -512,14 +512,14 @@ public class OverseerTest extends SolrTe
       }
       Thread.sleep(50);
     }
-    fail("Illegal state, was:" + coreState + " expected:" + expectedState + "cloudState:" + reader.getCloudState());
+    fail("Illegal state, was:" + coreState + " expected:" + expectedState + "clusterState:" + reader.getClusterState());
   }
   
   private void verifyShardLeader(ZkStateReader reader, String collection, String shard, String expectedCore) throws InterruptedException, KeeperException {
     int maxIterations = 100;
     while(maxIterations-->0) {
-      reader.updateCloudState(true); // poll state
-      ZkNodeProps props =  reader.getCloudState().getLeader(collection, shard);
+      reader.updateClusterState(true); // poll state
+      ZkNodeProps props =  reader.getClusterState().getLeader(collection, shard);
       if(props!=null) {
         if(expectedCore.equals(props.get(ZkStateReader.CORE_NAME_PROP))) {
           return;
@@ -528,7 +528,7 @@ public class OverseerTest extends SolrTe
       Thread.sleep(100);
     }
     
-    assertEquals("Unexpected shard leader coll:" + collection + " shard:" + shard, expectedCore, (reader.getCloudState().getLeader(collection, shard)!=null)?reader.getCloudState().getLeader(collection, shard).get(ZkStateReader.CORE_NAME_PROP):null);
+    assertEquals("Unexpected shard leader coll:" + collection + " shard:" + shard, expectedCore, (reader.getClusterState().getLeader(collection, shard)!=null)?reader.getClusterState().getLeader(collection, shard).get(ZkStateReader.CORE_NAME_PROP):null);
   }
 
   @Test
@@ -562,35 +562,35 @@ public class OverseerTest extends SolrTe
       waitForCollections(reader, "collection1");
       verifyStatus(reader, ZkStateReader.RECOVERING);
 
-      int version = getCloudStateVersion(controllerClient);
+      int version = getClusterStateVersion(controllerClient);
       
       mockController.publishState("core1", ZkStateReader.ACTIVE, 1);
       
-      while(version == getCloudStateVersion(controllerClient));
+      while(version == getClusterStateVersion(controllerClient));
 
       verifyStatus(reader, ZkStateReader.ACTIVE);
-      version = getCloudStateVersion(controllerClient);
+      version = getClusterStateVersion(controllerClient);
       overseerClient.close();
       Thread.sleep(1000); //wait for overseer to get killed
 
       mockController.publishState("core1", ZkStateReader.RECOVERING, 1);
-      version = getCloudStateVersion(controllerClient);
+      version = getClusterStateVersion(controllerClient);
       
       overseerClient = electNewOverseer(server.getZkAddress());
 
-      while(version == getCloudStateVersion(controllerClient));
+      while(version == getClusterStateVersion(controllerClient));
 
       verifyStatus(reader, ZkStateReader.RECOVERING);
       
-      assertEquals("Live nodes count does not match", 1, reader.getCloudState()
+      assertEquals("Live nodes count does not match", 1, reader.getClusterState()
           .getLiveNodes().size());
-      assertEquals("Shard count does not match", 1, reader.getCloudState()
+      assertEquals("Shard count does not match", 1, reader.getClusterState()
           .getSlice("collection1", "shard1").getShards().size());      
-      version = getCloudStateVersion(controllerClient);
+      version = getClusterStateVersion(controllerClient);
       mockController.publishState("core1", null,1);
-      while(version == getCloudStateVersion(controllerClient));
+      while(version == getClusterStateVersion(controllerClient));
       Thread.sleep(500);
-      assertFalse("collection1 should be gone after publishing the null state", reader.getCloudState().getCollections().contains("collection1"));
+      assertFalse("collection1 should be gone after publishing the null state", reader.getClusterState().getCollections().contains("collection1"));
     } finally {
       
       close(mockController);
@@ -740,15 +740,15 @@ public class OverseerTest extends SolrTe
 
       mockController.close();
 
-      int version = getCloudStateVersion(controllerClient);
+      int version = getClusterStateVersion(controllerClient);
       
       mockController = new MockZKController(server.getZkAddress(), "node1", "collection1");
       mockController.publishState("core1", ZkStateReader.RECOVERING, 1);
 
-      while (version == getCloudStateVersion(controllerClient));
+      while (version == getClusterStateVersion(controllerClient));
       
-      reader.updateCloudState(true);
-      CloudState state = reader.getCloudState();
+      reader.updateClusterState(true);
+      ClusterState state = reader.getClusterState();
       
       int numFound = 0;
       for (Map<String,Slice> collection : state.getCollectionStates().values()) {
@@ -758,7 +758,7 @@ public class OverseerTest extends SolrTe
           }
         }
       }
-      assertEquals("Shard was found in more than 1 times in CloudState", 1,
+      assertEquals("Shard was found more than once in ClusterState", 1,
           numFound);
     } finally {
       close(overseerClient);
@@ -800,7 +800,7 @@ public class OverseerTest extends SolrTe
 
       waitForCollections(reader, "collection1");
       
-      assertEquals("Slicecount does not match", 12, reader.getCloudState().getSlices("collection1").size());
+      assertEquals("Slicecount does not match", 12, reader.getClusterState().getSlices("collection1").size());
       
     } finally {
       close(overseerClient);
@@ -872,12 +872,12 @@ public class OverseerTest extends SolrTe
       queue.offer(ZkStateReader.toJSON(m));
       
       for(int i=0;i<100;i++) {
-        Slice s = reader.getCloudState().getSlice("collection1", "s1");
+        Slice s = reader.getClusterState().getSlice("collection1", "s1");
         if(s!=null && s.getShards().size()==3) break;
         Thread.sleep(100);
       }
-      assertNotNull(reader.getCloudState().getSlice("collection1", "s1"));
-      assertEquals(3, reader.getCloudState().getSlice("collection1", "s1").getShards().size());
+      assertNotNull(reader.getClusterState().getSlice("collection1", "s1"));
+      assertEquals(3, reader.getClusterState().getSlice("collection1", "s1").getShards().size());
     } finally {
       close(overseerClient);
       close(zkClient);
@@ -898,18 +898,22 @@ public class OverseerTest extends SolrTe
     }
   }
   
-  private int getCloudStateVersion(SolrZkClient controllerClient)
+  private int getClusterStateVersion(SolrZkClient controllerClient)
       throws KeeperException, InterruptedException {
     return controllerClient.exists(ZkStateReader.CLUSTER_STATE, null, false).getVersion();
   }
 
 
   private SolrZkClient electNewOverseer(String address) throws InterruptedException,
-      TimeoutException, IOException, KeeperException, ParserConfigurationException, SAXException {
-    SolrZkClient zkClient  = new SolrZkClient(address, TIMEOUT);
+ TimeoutException, IOException,
+      KeeperException, ParserConfigurationException, SAXException {
+    SolrZkClient zkClient = new SolrZkClient(address, TIMEOUT);
     ZkStateReader reader = new ZkStateReader(zkClient);
     LeaderElector overseerElector = new LeaderElector(zkClient);
-    ElectionContext ec = new OverseerElectionContext(new HttpShardHandlerFactory().getShardHandler(), "/admin/cores", address.replaceAll("/", "_"), reader);
+    // TODO: close Overseer
+    Overseer overseer = new Overseer(
+        new HttpShardHandlerFactory().getShardHandler(), "/admin/cores", reader);
+    ElectionContext ec = new OverseerElectionContext(zkClient, overseer, address.replaceAll("/", "_"));
     overseerElector.setup(ec);
     overseerElector.joinElection(ec);
     return zkClient;

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java Sat Aug  4 21:26:10 2012
@@ -157,7 +157,7 @@ public class SyncSliceTest extends FullS
     // we are careful to make sure the downed node is no longer in the state,
     // because on some systems (especially freebsd w/ blackhole enabled), trying
     // to talk to a downed node causes grief
-    waitToSeeDownInCloudState(leaderJetty, jetties);
+    waitToSeeDownInClusterState(leaderJetty, jetties);
 
     waitForThingsToLevelOut();
     
@@ -217,7 +217,7 @@ public class SyncSliceTest extends FullS
     // kill the current leader
     chaosMonkey.killJetty(leaderJetty);
     
-    waitToSeeDownInCloudState(leaderJetty, jetties);
+    waitToSeeDownInClusterState(leaderJetty, jetties);
     
     Thread.sleep(4000);
     
@@ -248,7 +248,7 @@ public class SyncSliceTest extends FullS
     return skipServers;
   }
 
-  private void waitToSeeDownInCloudState(CloudJettyRunner leaderJetty,
+  private void waitToSeeDownInClusterState(CloudJettyRunner leaderJetty,
       Set<CloudJettyRunner> jetties) throws InterruptedException {
 
     for (CloudJettyRunner cjetty : jetties) {

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java Sat Aug  4 21:26:10 2012
@@ -202,7 +202,7 @@ public class ZkControllerTest extends So
       
       assertNotNull(reader.getLeaderUrl("collection1", "shard1", 15000));
       
-      assertEquals("Shard(s) missing from cloudstate", 2, zkController.getZkStateReader().getCloudState().getSlice("collection1", "shard1").getShards().size());
+      assertEquals("Shard(s) missing from cloudstate", 2, zkController.getZkStateReader().getClusterState().getSlice("collection1", "shard1").getShards().size());
       
       // unregister current leader
       final ZkNodeProps shard1LeaderProps = reader.getLeaderProps(
@@ -224,10 +224,10 @@ public class ZkControllerTest extends So
           reader.getLeaderUrl("collection1", "shard1", 15000));
 
       for(int i=0;i<30;i++) {
-        if(zkController.getZkStateReader().getCloudState().getSlice("collection1", "shard1").getShards().size()==1) break; 
+        if(zkController.getZkStateReader().getClusterState().getSlice("collection1", "shard1").getShards().size()==1) break; 
         Thread.sleep(500);
       }
-      assertEquals("shard was not unregistered", 1, zkController.getZkStateReader().getCloudState().getSlice("collection1", "shard1").getShards().size());
+      assertEquals("shard was not unregistered", 1, zkController.getZkStateReader().getClusterState().getSlice("collection1", "shard1").getShards().size());
     } finally {
       System.clearProperty("solrcloud.skip.autorecovery");
       System.clearProperty(ZkStateReader.NUM_SHARDS_PROP);

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/core/CoreContainerCoreInitFailuresTest.java Sat Aug  4 21:26:10 2012
@@ -123,6 +123,11 @@ public class CoreContainerCoreInitFailur
   }
   
   public void testFlowBadFromStart() throws Exception {
+    // TODO: even if we close all solr cores in the container, there is still a leaked dir?
+    // maybe from one that didnt load right?
+    
+    // TODO: make SolrCore closeable since its has close()
+    System.setProperty("solr.directoryFactory", "org.apache.solr.core.SimpleFSDirectoryFactory");
     
     // reused state
     Map<String,Exception> failures = null;

Modified: lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/core/src/test/org/apache/solr/update/SolrCmdDistributorTest.java Sat Aug  4 21:26:10 2012
@@ -24,13 +24,16 @@ import java.util.List;
 
 import org.apache.solr.BaseDistributedSearchTestCase;
 import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
+import org.apache.solr.client.solrj.request.LukeRequest;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.update.SolrCmdDistributor.Node;
 import org.apache.solr.update.SolrCmdDistributor.Response;
 import org.apache.solr.update.SolrCmdDistributor.StdNode;
@@ -40,7 +43,7 @@ public class SolrCmdDistributorTest exte
   
   public SolrCmdDistributorTest() {
     fixShardCount = true;
-    shardCount = 1;
+    shardCount = 4;
     stress = 0;
   }
   
@@ -80,7 +83,7 @@ public class SolrCmdDistributorTest exte
   
   @Override
   public void doTest() throws Exception {
-    //del("*:*");
+    del("*:*");
     
     SolrCmdDistributor cmdDistrib = new SolrCmdDistributor(8);
     
@@ -159,6 +162,7 @@ public class SolrCmdDistributorTest exte
     
     assertEquals(response.errors.toString(), 0, response.errors.size());
     
+    
     results = controlClient.query(new SolrQuery("*:*")).getResults();
     numFound = results.getNumFound();
     assertEquals(results.toString(), 2, numFound);
@@ -166,5 +170,48 @@ public class SolrCmdDistributorTest exte
     numFound = client.query(new SolrQuery("*:*")).getResults()
         .getNumFound();
     assertEquals(results.toString(), 2, numFound);
+    
+    // debug stuff
+    for (SolrServer c : clients) {
+      c.optimize();
+      // distrib optimize is not working right yet, so call it on each client
+      //System.out.println(clients.get(0).request(new LukeRequest()));
+    }
+    
+    int id = 5;
+    
+    cmdDistrib = new SolrCmdDistributor(8);
+    
+    nodes.clear();
+    int cnt = atLeast(200);
+    for (int i = 0; i < cnt; i++) {
+      nodes.clear();
+      for (SolrServer c : clients) {
+        if (random().nextBoolean()) {
+          continue;
+        }
+        HttpSolrServer httpClient = (HttpSolrServer) c;
+        nodeProps = new ZkNodeProps(ZkStateReader.BASE_URL_PROP,
+            httpClient.getBaseURL(), ZkStateReader.CORE_NAME_PROP, "");
+        nodes.add(new StdNode(new ZkCoreNodeProps(nodeProps)));
+
+      }
+      
+      cmd.solrDoc = sdoc("id", id++);
+      cmdDistrib.distribAdd(cmd, nodes, params);
+    }
+
+    cmdDistrib.finish();
+    
+    cmdDistrib.distribCommit(ccmd, nodes, params);
+    
+    for (SolrServer c : clients) {
+      NamedList<Object> resp = c.request(new LukeRequest());
+      System.out.println(resp);
+      assertEquals("SOLR-3428: We only did adds - there should be no deletes",
+          ((NamedList<Object>) resp.get("index")).get("numDocs"),
+          ((NamedList<Object>) resp.get("index")).get("maxDoc"));
+    }
+
   }
 }

Modified: lucene/dev/branches/pforcodec_3892/solr/licenses/xercesImpl-2.9.1.jar.sha1
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/licenses/xercesImpl-2.9.1.jar.sha1?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/licenses/xercesImpl-2.9.1.jar.sha1 (original)
+++ lucene/dev/branches/pforcodec_3892/solr/licenses/xercesImpl-2.9.1.jar.sha1 Sat Aug  4 21:26:10 2012
@@ -1 +1 @@
-7bc7e49ddfe4fb5f193ed37ecc96c12292c8ceb6
\ No newline at end of file
+7bc7e49ddfe4fb5f193ed37ecc96c12292c8ceb6

Modified: lucene/dev/branches/pforcodec_3892/solr/solrj/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/solrj/ivy.xml?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/solrj/ivy.xml (original)
+++ lucene/dev/branches/pforcodec_3892/solr/solrj/ivy.xml Sat Aug  4 21:26:10 2012
@@ -20,7 +20,7 @@
     <info organisation="org.apache.solr" module="solrj"/>
 
     <dependencies>
-      <dependency org="org.apache.zookeeper" name="zookeeper" rev="3.3.5" transitive="false"/>
+      <dependency org="org.apache.zookeeper" name="zookeeper" rev="3.3.6" transitive="false"/>
       <dependency org="org.slf4j" name="log4j-over-slf4j" rev="1.6.4" transitive="false"/>
       <dependency org="org.apache.httpcomponents" name="httpcore" rev="4.1.4" transitive="false"/>
       <dependency org="org.apache.httpcomponents" name="httpclient" rev="4.1.3" transitive="false"/>

Modified: lucene/dev/branches/pforcodec_3892/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java?rev=1369470&r1=1369469&r2=1369470&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java (original)
+++ lucene/dev/branches/pforcodec_3892/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java Sat Aug  4 21:26:10 2012
@@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.util.ClientUtils;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -139,7 +139,7 @@ public class CloudSolrServer extends Sol
 
     // TODO: if you can hash here, you could favor the shard leader
     
-    CloudState cloudState = zkStateReader.getCloudState();
+    ClusterState clusterState = zkStateReader.getClusterState();
 
     SolrParams reqParams = request.getParams();
     if (reqParams == null) {
@@ -159,12 +159,12 @@ public class CloudSolrServer extends Sol
     Map<String,Slice> slices = new HashMap<String,Slice>();
     for (int i = 0; i < collectionList.size(); i++) {
       String coll= collectionList.get(i);
-      ClientUtils.appendMap(coll, slices, cloudState.getSlices(coll));
+      ClientUtils.appendMap(coll, slices, clusterState.getSlices(coll));
     }
 
-    Set<String> liveNodes = cloudState.getLiveNodes();
+    Set<String> liveNodes = clusterState.getLiveNodes();
 
-    // IDEA: have versions on various things... like a global cloudState version
+    // IDEA: have versions on various things... like a global clusterState version
     // or shardAddressVersion (which only changes when the shards change)
     // to allow caching.