You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/10/04 17:08:35 UTC

svn commit: r1629403 [4/4] - in /lucene/dev/branches/lucene5969: ./ dev-tools/ dev-tools/scripts/ lucene/ lucene/backward-codecs/ lucene/backward-codecs/src/test/org/apache/lucene/index/ lucene/core/ lucene/core/src/java/org/apache/lucene/document/ luc...

Modified: lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java (original)
+++ lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java Sat Oct  4 15:08:32 2014
@@ -302,36 +302,23 @@ public class HttpPartitionTest extends A
         cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
     assertEquals(expectedNewLeaderCoreNodeName, currentLeader.getName());
 
+    // TODO: This test logic seems to be timing dependent and fails on Jenkins
+    // need to come up with a better approach
     log.info("Sending doc 2 to old leader "+leader.getName());
     try {
       leaderSolr.add(doc);
       leaderSolr.shutdown();
 
-      Replica oldLeaderInRecovery = null;
-      for (Replica next : getActiveOrRecoveringReplicas(testCollectionName, "shard1")) {
-        if (next.getName().equals(leader.getName()) &&
-            ZkStateReader.RECOVERING.equals(next.getStr(ZkStateReader.STATE_PROP)))
-        {
-          oldLeaderInRecovery = next;
-          break;
-        }
-      }
-
-      // if the old leader is not active or recovering, the add should have failed
-      if (oldLeaderInRecovery != null) {
-        HttpSolrServer oldLeaderSolr = getHttpSolrServer(oldLeaderInRecovery, testCollectionName);
-        try {
-          assertDocExists(oldLeaderSolr, testCollectionName, "2");
-        } finally {
-          oldLeaderSolr.shutdown();
-        }
-      } else {
-        fail("Send doc 2 to old leader " + leader.getName() +
-            " should have failed! ClusterState: " + printClusterStateInfo(testCollectionName));
+      // if the add worked, then the doc must exist on the new leader
+      HttpSolrServer newLeaderSolr = getHttpSolrServer(currentLeader, testCollectionName);
+      try {
+        assertDocExists(newLeaderSolr, testCollectionName, "2");
+      } finally {
+        newLeaderSolr.shutdown();
       }
 
     } catch (SolrException exc) {
-      // this is expected ..
+      // this is ok provided the doc doesn't exist on the current leader
       leaderSolr = getHttpSolrServer(currentLeader, testCollectionName);
       try {
         leaderSolr.add(doc); // this should work

Modified: lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java (original)
+++ lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java Sat Oct  4 15:08:32 2014
@@ -19,20 +19,30 @@ package org.apache.solr.cloud;
 
 
 import com.google.common.collect.Lists;
+import org.apache.commons.lang.StringUtils;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrServer;
 import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.zookeeper.KeeperException;
 import org.junit.Before;
 
+import static org.apache.solr.cloud.OverseerCollectionProcessor.SLICE_UNIQUE;
+
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
 public class TestCollectionAPI extends AbstractFullDistribZkTestBase {
@@ -57,7 +67,7 @@ public class TestCollectionAPI extends A
   public void doTest() throws Exception {
     CloudSolrServer client = createCloudClient(null);
     try {
-      createCollection(null, COLLECTION_NAME, 2, 1, 1, client, null, "conf1");
+      createCollection(null, COLLECTION_NAME, 2, 2, 2, client, null, "conf1");
       createCollection(null, COLLECTION_NAME1, 1, 1, 1, client, null, "conf1");
     } finally {
       //remove collections
@@ -76,6 +86,7 @@ public class TestCollectionAPI extends A
     clusterStatusWithRouteKey();
     clusterStatusAliasTest();
     clusterStatusRolesTest();
+    replicaPropTest();
   }
 
   private void clusterStatusWithCollectionAndShard() throws IOException, SolrServerException {
@@ -281,4 +292,392 @@ public class TestCollectionAPI extends A
       client.shutdown();
     }
   }
+
+  private void replicaPropTest() throws Exception {
+    CloudSolrServer client = createCloudClient(null);
+    try {
+      client.connect();
+      Map<String, Slice> slices = client.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();
+      List<String> sliceList = new ArrayList<>(slices.keySet());
+      String c1_s1 = sliceList.get(0);
+      List<String> replicasList = new ArrayList<>(slices.get(c1_s1).getReplicasMap().keySet());
+      String c1_s1_r1 = replicasList.get(0);
+      String c1_s1_r2 = replicasList.get(1);
+
+      String c1_s2 = sliceList.get(1);
+      replicasList = new ArrayList<>(slices.get(c1_s2).getReplicasMap().keySet());
+      String c1_s2_r1 = replicasList.get(0);
+      String c1_s2_r2 = replicasList.get(1);
+
+
+      slices = client.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME1).getSlicesMap();
+      sliceList = new ArrayList<>(slices.keySet());
+      String c2_s1 = sliceList.get(0);
+      replicasList = new ArrayList<>(slices.get(c2_s1).getReplicasMap().keySet());
+      String c2_s1_r1 = replicasList.get(0);
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString());
+
+      // Insure we get error returns when omitting required parameters
+
+      missingParamsError(client, params);
+      params.set("collection", COLLECTION_NAME);
+      missingParamsError(client, params);
+      params.set("shard", c1_s1);
+      missingParamsError(client, params);
+      params.set("replica", c1_s1_r1);
+      missingParamsError(client, params);
+      params.set("property", "preferredLeader");
+      missingParamsError(client, params);
+      params.set("property.value", "true");
+
+      SolrRequest request = new QueryRequest(params);
+      request.setPath("/admin/collections");
+      client.request(request);
+
+      // The above should have set exactly one preferredleader...
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r2, "property.preferredLeader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r1, "property.preferredLeader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredLeader");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r2,
+          "property", "preferredLeader",
+          "property.value", "true");
+      // The preferred leader property for shard1 should have switched to the other replica.
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredLeader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r1, "property.preferredLeader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredLeader");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s2,
+          "replica", c1_s2_r1,
+          "property", "preferredLeader",
+          "property.value", "true");
+
+      // Now we should have a preferred leader in both shards...
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredLeader");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
+          "collection", COLLECTION_NAME1,
+          "shard", c2_s1,
+          "replica", c2_s1_r1,
+          "property", "preferredLeader",
+          "property.value", "true");
+
+      // Now we should have three preferred leaders.
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader", "true");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toString(),
+          "collection", COLLECTION_NAME1,
+          "shard", c2_s1,
+          "replica", c2_s1_r1,
+          "property", "preferredLeader");
+
+      // Now we should have two preferred leaders.
+      // But first we have to wait for the overseer to finish the action
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+      // Try adding an arbitrary property to one that has the leader property
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "testprop",
+          "property.value", "true");
+
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r2,
+          "property", "prop",
+          "property.value", "silly");
+
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "testprop",
+          "property.value", "nonsense",
+          SLICE_UNIQUE, "true");
+
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "nonsense");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "property.testprop",
+          "property.value", "true",
+          SLICE_UNIQUE, "false");
+
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.testprop", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "property.testprop");
+
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.testprop");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+      try {
+        doPropertyAction(client,
+            "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toString(),
+            "collection", COLLECTION_NAME,
+            "shard", c1_s1,
+            "replica", c1_s1_r1,
+            "property", "preferredLeader",
+            "property.value", "true",
+            SLICE_UNIQUE, "false");
+        fail("Should have thrown an exception, setting sliceUnique=false is not allowed for 'preferredLeader'.");
+      } catch (SolrException se) {
+        assertTrue("Should have received a specific error message",
+            se.getMessage().contains("with the sliceUnique parameter set to something other than 'true'"));
+      }
+
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.testprop");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.prop", "silly");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s2_r2, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+      verifyPropertyNotPresent(client, COLLECTION_NAME1, c2_s1_r1, "property.preferredleader");
+
+      Map<String, String> origProps = getProps(client, COLLECTION_NAME, c1_s1_r1,
+          "state", "core", "node_name", "base_url");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "state",
+          "property.value", "state_bad");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "core",
+          "property.value", "core_bad");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "node_name",
+          "property.value", "node_name_bad");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.ADDREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "base_url",
+          "property.value", "base_url_bad");
+
+      // The above should be on new proeprties.
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.state", "state_bad");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.core", "core_bad");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.node_name", "node_name_bad");
+      verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, "property.base_url", "base_url_bad");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "state");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "core");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "node_name");
+
+      doPropertyAction(client,
+          "action", CollectionParams.CollectionAction.DELETEREPLICAPROP.toLower(),
+          "collection", COLLECTION_NAME,
+          "shard", c1_s1,
+          "replica", c1_s1_r1,
+          "property", "base_url");
+
+      // They better not have been changed!
+      for (Map.Entry<String, String> ent : origProps.entrySet()) {
+        verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r1, ent.getKey(), ent.getValue());
+      }
+
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.state");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.core");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.node_name");
+      verifyPropertyNotPresent(client, COLLECTION_NAME, c1_s1_r1, "property.base_url");
+
+    } finally {
+      client.shutdown();
+    }
+  }
+
+  private void doPropertyAction(CloudSolrServer client, String... paramsIn) throws IOException, SolrServerException {
+    assertTrue("paramsIn must be an even multiple of 2, it is: " + paramsIn.length, (paramsIn.length % 2) == 0);
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    for (int idx = 0; idx < paramsIn.length; idx += 2) {
+      params.set(paramsIn[idx], paramsIn[idx + 1]);
+    }
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    client.request(request);
+
+  }
+
+  private void verifyPropertyNotPresent(CloudSolrServer client, String collectionName, String replicaName,
+                                        String property)
+      throws KeeperException, InterruptedException {
+    ClusterState clusterState = null;
+    Replica replica = null;
+    for (int idx = 0; idx < 300; ++idx) {
+      client.getZkStateReader().updateClusterState(true);
+      clusterState = client.getZkStateReader().getClusterState();
+      replica = clusterState.getReplica(collectionName, replicaName);
+      if (replica == null) {
+        fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
+      }
+      if (StringUtils.isBlank(replica.getStr(property))) return;
+      Thread.sleep(100);
+    }
+    fail("Property " + property + " not set correctly for collection/replica pair: " +
+        collectionName + "/" + replicaName + ". Replica props: " + replica.getProperties().toString() +
+        ". Cluster state is " + clusterState.toString());
+
+  }
+
+  // The params are triplets,
+  // collection
+  // shard
+  // replica
+  private void verifyPropertyVal(CloudSolrServer client, String collectionName,
+                                 String replicaName, String property, String val)
+      throws InterruptedException, KeeperException {
+    Replica replica = null;
+    ClusterState clusterState = null;
+
+    for (int idx = 0; idx < 300; ++idx) { // Keep trying while Overseer writes the ZK state for up to 30 seconds.
+      client.getZkStateReader().updateClusterState(true);
+      clusterState = client.getZkStateReader().getClusterState();
+      replica = clusterState.getReplica(collectionName, replicaName);
+      if (replica == null) {
+        fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
+      }
+      if (StringUtils.equals(val, replica.getStr(property))) return;
+      Thread.sleep(100);
+    }
+
+    fail("Property '" + property + "' with value " + replica.getStr(property) +
+        " not set correctly for collection/replica pair: " + collectionName + "/" + replicaName + " property map is " +
+        replica.getProperties().toString() + ".");
+
+  }
+
+  // Expects the map will have keys, but blank values.
+  private Map<String, String> getProps(CloudSolrServer client, String collectionName, String replicaName, String... props) throws KeeperException, InterruptedException {
+    client.getZkStateReader().updateClusterState(true);
+    ClusterState clusterState = client.getZkStateReader().getClusterState();
+    Replica replica = clusterState.getReplica(collectionName, replicaName);
+    if (replica == null) {
+      fail("Could not find collection/replica pair! " + collectionName + "/" + replicaName);
+    }
+    Map<String, String> propMap = new HashMap<>();
+    for (String prop : props) {
+      propMap.put(prop, replica.getStr(prop));
+    }
+    return propMap;
+  }
+  private void missingParamsError(CloudSolrServer client, ModifiableSolrParams origParams)
+      throws IOException, SolrServerException {
+
+    SolrRequest request;
+    try {
+      request = new QueryRequest(origParams);
+      request.setPath("/admin/collections");
+      client.request(request);
+      fail("Should have thrown a SolrException due to lack of a required parameter.");
+    } catch (SolrException se) {
+      assertTrue("Should have gotten a specific message back mentioning 'missing required parameter'. Got: " + se.getMessage(),
+          se.getMessage().toLowerCase(Locale.ROOT).contains("missing required parameter:"));
+    }
+  }
 }

Modified: lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java (original)
+++ lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/core/ExitableDirectoryReaderTest.java Sat Oct  4 15:08:32 2014
@@ -29,13 +29,13 @@ import org.junit.Test;
  */
 public class ExitableDirectoryReaderTest extends SolrTestCaseJ4 {
   
-  static int NUM_DOCS_PER_TYPE = 2000;
+  static int NUM_DOCS_PER_TYPE = 100;
   static final String assertionString = "//result[@numFound='"+ (NUM_DOCS_PER_TYPE - 1) + "']";
 
   @BeforeClass
   public static void beforeClass() throws Exception {
     System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
-    initCore("solrconfig-nocache.xml", "schema12.xml");
+    initCore("solrconfig-nocache-with-delaying-searchcomponent.xml", "schema12.xml");
     createIndex();
   }
 
@@ -62,13 +62,13 @@ public class ExitableDirectoryReaderTest
         , SolrException.ErrorCode.BAD_REQUEST
     );
 
-    assertQ(req("q","name:a*", "indent","true", "timeAllowed","1000"), assertionString);
+    assertQ(req("q","name:a*", "indent","true", "timeAllowed","10000"), assertionString);
 
     assertQEx("", req("q","name:a*", "indent","true", "timeAllowed","1")
         , SolrException.ErrorCode.BAD_REQUEST
     );
 
-    assertQ(req("q","name:b*", "indent","true", "timeAllowed","1000"), assertionString);
+    assertQ(req("q","name:b*", "indent","true", "timeAllowed","10000"), assertionString);
 
     assertQ(req("q","name:b*", "indent","true", "timeAllowed",Long.toString(Long.MAX_VALUE)), assertionString);
 
@@ -79,11 +79,11 @@ public class ExitableDirectoryReaderTest
   
   @Test
   public void testQueriesOnDocsWithMultipleTerms() {
-    assertQ(req("q","name:dummy", "indent","true", "timeAllowed","1000"), assertionString);
+    assertQ(req("q","name:dummy", "indent","true", "timeAllowed","10000"), assertionString);
 
     // This should pass even though this may take more than the 'timeAllowed' time, it doesn't take long
     // to iterate over 1 term (dummy).
-    assertQ(req("q","name:dummy", "indent","true", "timeAllowed","1000"), assertionString);
+    assertQ(req("q","name:dummy", "indent","true", "timeAllowed","10000"), assertionString);
 
     assertQEx("", req("q","name:doc*", "indent","true", "timeAllowed","1")
         , SolrException.ErrorCode.BAD_REQUEST

Modified: lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java (original)
+++ lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java Sat Oct  4 15:08:32 2014
@@ -17,8 +17,15 @@ package org.apache.solr.schema;
  */
 
 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.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.util.BaseTestHarness;
 import org.apache.solr.util.RESTfulServerProvider;
 import org.apache.solr.util.RestTestHarness;
@@ -27,6 +34,8 @@ import org.restlet.ext.servlet.ServerSer
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.zookeeper.data.Stat;
+
 import org.junit.BeforeClass;
 
 import java.util.ArrayList;
@@ -96,51 +105,65 @@ public class TestCloudManagedSchemaConcu
     }
   }
 
-  private static void addFieldPut(RestTestHarness publisher, String fieldName) throws Exception {
+  private static void addFieldPut(RestTestHarness publisher, String fieldName, int updateTimeoutSecs) throws Exception {
     final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
     String request = "/schema/fields/" + fieldName + "?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.put(request, content);
     verifySuccess(request, response);
   }
 
-  private static void addFieldPost(RestTestHarness publisher, String fieldName) throws Exception {
+  private static void addFieldPost(RestTestHarness publisher, String fieldName, int updateTimeoutSecs) throws Exception {
     final String content = "[{\"name\":\""+fieldName+"\",\"type\":\"text\",\"stored\":\"false\"}]";
     String request = "/schema/fields/?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.post(request, content);
     verifySuccess(request, response);
   }
 
-  private static void addDynamicFieldPut(RestTestHarness publisher, String dynamicFieldPattern) throws Exception {
+  private static void addDynamicFieldPut(RestTestHarness publisher, String dynamicFieldPattern, int updateTimeoutSecs) throws Exception {
     final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
     String request = "/schema/dynamicfields/" + dynamicFieldPattern + "?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.put(request, content);
     verifySuccess(request, response);
   }
 
-  private static void addDynamicFieldPost(RestTestHarness publisher, String dynamicFieldPattern) throws Exception {
+  private static void addDynamicFieldPost(RestTestHarness publisher, String dynamicFieldPattern, int updateTimeoutSecs) throws Exception {
     final String content = "[{\"name\":\""+dynamicFieldPattern+"\",\"type\":\"text\",\"stored\":\"false\"}]";
     String request = "/schema/dynamicfields/?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.post(request, content);
     verifySuccess(request, response);
   }
 
-  private static void copyField(RestTestHarness publisher, String source, String dest) throws Exception {
+  private static void copyField(RestTestHarness publisher, String source, String dest, int updateTimeoutSecs) throws Exception {
     final String content = "[{\"source\":\""+source+"\",\"dest\":[\""+dest+"\"]}]";
     String request = "/schema/copyfields/?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.post(request, content);
     verifySuccess(request, response);
   }
 
-  private static void addFieldTypePut(RestTestHarness publisher, String typeName) throws Exception {
+  private static void addFieldTypePut(RestTestHarness publisher, String typeName, int updateTimeoutSecs) throws Exception {
     final String content = "{\"class\":\"solr.TrieIntField\"}";
     String request = "/schema/fieldtypes/" + typeName + "?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.put(request, content);
     verifySuccess(request, response);
   }
 
-  private static void addFieldTypePost(RestTestHarness publisher, String typeName) throws Exception {
+  private static void addFieldTypePost(RestTestHarness publisher, String typeName, int updateTimeoutSecs) throws Exception {
     final String content = "[{\"name\":\""+typeName+"\",\"class\":\"solr.TrieIntField\"}]";
     String request = "/schema/fieldtypes/?wt=xml";
+    if (updateTimeoutSecs > 0)
+      request += "&updateTimeoutSecs="+updateTimeoutSecs;
     String response = publisher.post(request, content);
     verifySuccess(request, response);
   }
@@ -220,6 +243,7 @@ public class TestCloudManagedSchemaConcu
 
   @Override
   public void doTest() throws Exception {
+    verifyWaitForSchemaUpdateToPropagate();
     setupHarnesses();
     concurrentOperationsTest();
     schemaLockTest();
@@ -244,23 +268,23 @@ public class TestCloudManagedSchemaConcu
     PUT_AddField {
       @Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
         String fieldname = PUT_FIELDNAME + info.numAddFieldPuts++;
-        addFieldPut(publisher, fieldname);
+        addFieldPut(publisher, fieldname, 15);
       }
     },
     POST_AddField {
       @Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
         String fieldname = POST_FIELDNAME + info.numAddFieldPosts++;
-        addFieldPost(publisher, fieldname);
+        addFieldPost(publisher, fieldname, 15);
       }
     },
     PUT_AddDynamicField {
       @Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
-        addDynamicFieldPut(publisher, PUT_DYNAMIC_FIELDNAME + info.numAddDynamicFieldPuts++ + "_*");
+        addDynamicFieldPut(publisher, PUT_DYNAMIC_FIELDNAME + info.numAddDynamicFieldPuts++ + "_*", 15);
       }
     },
     POST_AddDynamicField {
       @Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
-        addDynamicFieldPost(publisher, POST_DYNAMIC_FIELDNAME + info.numAddDynamicFieldPosts++ + "_*");
+        addDynamicFieldPost(publisher, POST_DYNAMIC_FIELDNAME + info.numAddDynamicFieldPosts++ + "_*", 15);
       }
     },
     POST_AddCopyField {
@@ -273,7 +297,7 @@ public class TestCloudManagedSchemaConcu
           sourceField = "name";
         } else if (sourceType == 1) { // newly created
           sourceField = "copySource" + fieldNum;
-          addFieldPut(publisher, sourceField);
+          addFieldPut(publisher, sourceField, 15);
         } else { // dynamic
           sourceField = "*_dynamicSource" + fieldNum + "_t";
           // * only supported if both src and dst use it
@@ -286,23 +310,23 @@ public class TestCloudManagedSchemaConcu
             destField = "title";
           } else { // newly created
             destField = "copyDest" + fieldNum;
-            addFieldPut(publisher, destField);
+            addFieldPut(publisher, destField, 15);
           }
         }
-        copyField(publisher, sourceField, destField);
+        copyField(publisher, sourceField, destField, 15);
         info.copyFields.add(new CopyFieldInfo(sourceField, destField));
       }
     },
     PUT_AddFieldType {
       @Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
         String typeName = PUT_FIELDTYPE + info.numAddFieldTypePuts++;
-        addFieldTypePut(publisher, typeName);
+        addFieldTypePut(publisher, typeName, 15);
       }
     },
     POST_AddFieldType {
       @Override public void execute(RestTestHarness publisher, int fieldNum, Info info) throws Exception {
         String typeName = POST_FIELDTYPE + info.numAddFieldTypePosts++;
-        addFieldTypePost(publisher, typeName);
+        addFieldTypePost(publisher, typeName, 15);
       }
     };
 
@@ -315,6 +339,95 @@ public class TestCloudManagedSchemaConcu
     }
   }
 
+  private void verifyWaitForSchemaUpdateToPropagate() throws Exception {
+    String testCollectionName = "collection1";
+
+    ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
+    Replica shard1Leader = clusterState.getLeader(testCollectionName, "shard1");
+    final String coreUrl = (new ZkCoreNodeProps(shard1Leader)).getCoreUrl();
+    assertNotNull(coreUrl);
+
+    RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
+      public String getBaseURL() {
+        return coreUrl.endsWith("/") ? coreUrl.substring(0, coreUrl.length()-1) : coreUrl;
+      }
+    });
+
+    addFieldTypePut(harness, "fooInt", 15);
+
+    // go into ZK to get the version of the managed schema after the update
+    SolrZkClient zkClient = cloudClient.getZkStateReader().getZkClient();
+    Stat stat = new Stat();
+    String znodePath = "/configs/conf1/managed-schema";
+    byte[] managedSchemaBytes = zkClient.getData(znodePath, null, stat, false);
+    int schemaZkVersion = stat.getVersion();
+
+    // now loop over all replicas and verify each has the same schema version
+    Replica randomReplicaNotLeader = null;
+    for (Slice slice : clusterState.getActiveSlices(testCollectionName)) {
+      for (Replica replica : slice.getReplicas()) {
+        validateZkVersion(replica, schemaZkVersion, 0, false);
+
+        // save a random replica to test zk watcher behavior
+        if (randomReplicaNotLeader == null && !replica.getName().equals(shard1Leader.getName()))
+          randomReplicaNotLeader = replica;
+      }
+    }
+    assertNotNull(randomReplicaNotLeader);
+
+    // now update the data and then verify the znode watcher fires correctly
+    // before an after a zk session expiration (see SOLR-6249)
+    zkClient.setData(znodePath, managedSchemaBytes, schemaZkVersion, false);
+    stat = new Stat();
+    managedSchemaBytes = zkClient.getData(znodePath, null, stat, false);
+    int updatedSchemaZkVersion = stat.getVersion();
+    assertTrue(updatedSchemaZkVersion > schemaZkVersion);
+    validateZkVersion(randomReplicaNotLeader, updatedSchemaZkVersion, 2, true);
+
+    // ok - looks like the watcher fired correctly on the replica
+    // now, expire that replica's zk session and then verify the watcher fires again (after reconnect)
+    JettySolrRunner randomReplicaJetty =
+        getJettyOnPort(getReplicaPort(randomReplicaNotLeader));
+    assertNotNull(randomReplicaJetty);
+    chaosMonkey.expireSession(randomReplicaJetty);
+
+    // update the data again to cause watchers to fire
+    zkClient.setData(znodePath, managedSchemaBytes, updatedSchemaZkVersion, false);
+    stat = new Stat();
+    managedSchemaBytes = zkClient.getData(znodePath, null, stat, false);
+    updatedSchemaZkVersion = stat.getVersion();
+    // give up to 10 secs for the replica to recover after zk session loss and see the update
+    validateZkVersion(randomReplicaNotLeader, updatedSchemaZkVersion, 10, true);
+  }
+
+  /**
+   * Sends a GET request to get the zk schema version from a specific replica.
+   */
+  protected void validateZkVersion(Replica replica, int schemaZkVersion, int waitSecs, boolean retry) throws Exception {
+    final String replicaUrl = (new ZkCoreNodeProps(replica)).getCoreUrl();
+    RestTestHarness testHarness = new RestTestHarness(new RESTfulServerProvider() {
+      public String getBaseURL() {
+        return replicaUrl.endsWith("/") ? replicaUrl.substring(0, replicaUrl.length()-1) : replicaUrl;
+      }
+    });
+
+    long waitMs = waitSecs * 1000L;
+    if (waitMs > 0) Thread.sleep(waitMs); // wait a moment for the zk watcher to fire
+
+    try {
+      testHarness.validateQuery("/schema/zkversion?wt=xml", "//zkversion=" + schemaZkVersion);
+    } catch (Exception exc) {
+      if (retry) {
+        // brief wait before retrying
+        Thread.sleep(waitMs > 0 ? waitMs : 2000L);
+
+        testHarness.validateQuery("/schema/zkversion?wt=xml", "//zkversion=" + schemaZkVersion);
+      } else {
+        throw exc;
+      }
+    }
+  }
+
   private void concurrentOperationsTest() throws Exception {
     
     // First, add a bunch of fields and dynamic fields via PUT and POST, as well as copyFields,
@@ -405,7 +518,9 @@ public class TestCloudManagedSchemaConcu
     }
     public void run() {
       try {
-        addFieldPut(harness, fieldName);
+        // don't have the client side wait for all replicas to see the update or that defeats the purpose
+        // of testing the locking support on the server-side
+        addFieldPut(harness, fieldName, -1);
       } catch (Exception e) {
         // log.error("###ACTUAL FAILURE!");
         throw new RuntimeException(e);
@@ -420,7 +535,7 @@ public class TestCloudManagedSchemaConcu
     }
     public void run() {
       try {
-        addFieldPost(harness, fieldName);
+        addFieldPost(harness, fieldName, -1);
       } catch (Exception e) {
         // log.error("###ACTUAL FAILURE!");
         throw new RuntimeException(e);
@@ -435,7 +550,7 @@ public class TestCloudManagedSchemaConcu
     }
     public void run() {
       try {
-        addFieldTypePut(harness, fieldName);
+        addFieldTypePut(harness, fieldName, -1);
       } catch (Exception e) {
         // log.error("###ACTUAL FAILURE!");
         throw new RuntimeException(e);
@@ -450,7 +565,7 @@ public class TestCloudManagedSchemaConcu
     }
     public void run() {
       try {
-        addFieldTypePost(harness, fieldName);
+        addFieldTypePost(harness, fieldName, -1);
       } catch (Exception e) {
         // log.error("###ACTUAL FAILURE!");
         throw new RuntimeException(e);
@@ -465,7 +580,7 @@ public class TestCloudManagedSchemaConcu
     }
     public void run() {
       try {
-        addFieldPut(harness, fieldName);
+        addFieldPut(harness, fieldName, -1);
       } catch (Exception e) {
         // log.error("###ACTUAL FAILURE!");
         throw new RuntimeException(e);
@@ -480,7 +595,7 @@ public class TestCloudManagedSchemaConcu
     }
     public void run() {
       try {
-        addFieldPost(harness, fieldName);
+        addFieldPost(harness, fieldName, -1);
       } catch (Exception e) {
         // log.error("###ACTUAL FAILURE!");
         throw new RuntimeException(e);

Modified: lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java (original)
+++ lucene/dev/branches/lucene5969/solr/core/src/test/org/apache/solr/update/AddBlockUpdateTest.java Sat Oct  4 15:08:32 2014
@@ -5,6 +5,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermRangeFilter;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.join.FixedBitSetCachingWrapperFilter;
 import org.apache.lucene.search.join.ScoreMode;
 import org.apache.lucene.search.join.ToParentBlockJoinQuery;
 import org.apache.solr.SolrTestCaseJ4;
@@ -34,6 +35,7 @@ import javax.xml.stream.XMLStreamReader;
 
 
 
+
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -566,8 +568,8 @@ public class AddBlockUpdateTest extends 
   
   protected ToParentBlockJoinQuery join(final String childTerm) {
     return new ToParentBlockJoinQuery(
-        new TermQuery(new Term(child, childTerm)), new TermRangeFilter(parent,
-            null, null, false, false), ScoreMode.None);
+        new TermQuery(new Term(child, childTerm)), new FixedBitSetCachingWrapperFilter(new TermRangeFilter(parent,
+            null, null, false, false)), ScoreMode.None);
   }
   
   private Collection<? extends Callable<Void>> callables(List<Document> blocks) {

Modified: lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Sat Oct  4 15:08:32 2014
@@ -69,6 +69,8 @@ public class ZkStateReader implements Cl
   public static final String SHARD_PARENT_PROP = "shard_parent";
   public static final String NUM_SHARDS_PROP = "numShards";
   public static final String LEADER_PROP = "leader";
+  public static final String PROPERTY_PROP = "property";
+  public static final String PROPERTY_VALUE_PROP = "property.value";
   
   public static final String COLLECTIONS_ZKNODE = "/collections";
   public static final String LIVE_NODES_ZKNODE = "/live_nodes";
@@ -102,8 +104,7 @@ public class ZkStateReader implements Cl
   public static final String LEADER_ELECT_ZKNODE = "/leader_elect";
 
   public static final String SHARD_LEADERS_ZKNODE = "leaders";
-  public static final String ONLY_IF_DOWN = "onlyIfDown";
-  
+
   private final Set<String> watchedCollections = new HashSet<String>();
 
   /**These are collections which are actively watched by this  instance .

Modified: lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java (original)
+++ lucene/dev/branches/lucene5969/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java Sat Oct  4 15:08:32 2014
@@ -46,7 +46,9 @@ public interface CollectionParams 
     ADDREPLICA,
     OVERSEERSTATUS,
     LIST,
-    CLUSTERSTATUS;
+    CLUSTERSTATUS,
+    ADDREPLICAPROP,
+    DELETEREPLICAPROP;
     
     public static CollectionAction get( String p )
     {

Modified: lucene/dev/branches/lucene5969/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1629403&r1=1629402&r2=1629403&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/branches/lucene5969/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Sat Oct  4 15:08:32 2014
@@ -549,6 +549,14 @@ public abstract class AbstractFullDistri
     return jetty;
   }
 
+  protected int getReplicaPort(Replica replica) {
+    String replicaNode = replica.getNodeName();
+    String tmp = replicaNode.substring(replicaNode.indexOf(':')+1);
+    if (tmp.indexOf('_') != -1)
+      tmp = tmp.substring(0,tmp.indexOf('_'));
+    return Integer.parseInt(tmp);
+  }
+
   protected JettySolrRunner getJettyOnPort(int port) {
     JettySolrRunner theJetty = null;
     for (JettySolrRunner jetty : jettys) {