You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/04/01 03:40:24 UTC

[GitHub] [lucene-solr] madrob commented on a change in pull request #2479: SOLR-15288 Replicas stay DOWN after a new node is restarted when using the same directory

madrob commented on a change in pull request #2479:
URL: https://github.com/apache/lucene-solr/pull/2479#discussion_r605251098



##########
File path: solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
##########
@@ -24,19 +24,28 @@
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.PerReplicaStates;
 import org.apache.solr.common.cloud.PerReplicaStatesOps;
 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.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.cloud.overseer.SliceMutator.getZkClient;
+
 public class NodeMutator {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  protected SolrZkClient zkClient;
+  public NodeMutator(SolrCloudManager cloudManager) {
+    zkClient = getZkClient(cloudManager);

Review comment:
       please use the qualified classname instead of a static import. easier to find during review.

##########
File path: solr/core/src/test/org/apache/solr/cloud/NodeMutatorTest.java
##########
@@ -43,7 +43,7 @@
 
   @Test
   public void downNodeReportsAllImpactedCollectionsAndNothingElse() throws IOException {
-    NodeMutator nm = new NodeMutator();
+    NodeMutator nm = new NodeMutator(null);

Review comment:
       should we add a new constructor instead?

##########
File path: solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
##########
@@ -1099,7 +1102,7 @@ public void testPerReplicaStateCollection() throws Exception {
     c.forEachReplica((s, replica) -> assertNotNull(replica.getReplicaState()));
     PerReplicaStates prs = PerReplicaStates.fetch(ZkStateReader.getCollectionPath(testCollection), cluster.getZkClient(), null);
     assertEquals(4, prs.states.size());
-
+    JettySolrRunner jsr = cluster.startJettySolrRunner();

Review comment:
       unused store? shouldn't the cluster already be running at this point?

##########
File path: solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientTest.java
##########
@@ -1120,4 +1123,63 @@ public void testPerReplicaStateCollection() throws Exception {
     assertEquals(4, prs.states.size());
   }
 
+  public void testPRSRestart() throws Exception {
+    String testCollection = "prs_restart_test";
+    MiniSolrCloudCluster cluster1 =
+        configureCluster(1)
+            .addConfig("conf1", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
+            .withJettyConfig(jetty -> jetty.enableV2(true))
+            .configure();
+    try {
+      CollectionAdminRequest.createCollection(testCollection, "conf1", 1, 1)
+          .setPerReplicaState(Boolean.TRUE)
+          .process(cluster.getSolrClient());
+      cluster1.waitForActiveCollection(testCollection, 1, 1);
+
+      DocCollection c = cluster1.getSolrClient().getZkStateReader().getCollection(testCollection);
+      c.forEachReplica((s, replica) -> assertNotNull(replica.getReplicaState()));
+      String collectionPath = ZkStateReader.getCollectionPath(testCollection);
+      PerReplicaStates prs = PerReplicaStates.fetch(collectionPath, cluster.getZkClient(), null);
+      assertEquals(1, prs.states.size());
+
+      JettySolrRunner jsr = cluster1.startJettySolrRunner();
+      assertEquals(2,cluster1.getJettySolrRunners().size());
+
+      // Now let's do an add replica
+      CollectionAdminRequest
+          .addReplicaToShard(testCollection, "shard1")
+          .process(cluster1.getSolrClient());
+      cluster1.waitForActiveCollection(testCollection, 1, 2);
+      prs = PerReplicaStates.fetch(collectionPath, cluster.getZkClient(), null);
+      assertEquals(2, prs.states.size());
+      c = cluster1.getSolrClient().getZkStateReader().getCollection(testCollection);
+      prs.states.forEachEntry((s, state) -> assertEquals(Replica.State.ACTIVE, state.state));
+
+      String replicaName = null;
+      for (Replica r : c.getSlice("shard1").getReplicas()) {

Review comment:
       getReplica(BiPredicate) might be cleaner?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org