You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2021/11/24 10:13:20 UTC

[lucene-solr] branch branch_8_11 updated: SOLR-15794: Switching a PRS collection from true -> false -> true results in INACTIVE replicas

This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch branch_8_11
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8_11 by this push:
     new 5d26b61  SOLR-15794: Switching a PRS collection from true -> false -> true results in INACTIVE replicas
5d26b61 is described below

commit 5d26b61854c9b8377bd5f8efacbc49c61cb71ae7
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Mon Nov 15 13:47:39 2021 +1100

    SOLR-15794: Switching a PRS collection from true -> false -> true results in INACTIVE replicas
---
 solr/CHANGES.txt                                   |  2 +
 .../apache/solr/common/cloud/PerReplicaStates.java |  4 ++
 .../solr/common/cloud/PerReplicaStatesOps.java     | 17 ++++++--
 .../cloud/PerReplicaStatesIntegrationTest.java     | 46 ++++++++++++++++++++++
 4 files changed, 66 insertions(+), 3 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index fb0e940..f2bf6f2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -21,6 +21,8 @@ Bug Fixes
 
 * SOLR-15774: Avoid weird off-by-one errors with Angular's 'chosen' select box directive for the security and schema-designer screens in Admin UI (Timothy Potter)
 
+* SOLR-15794: Switching a PRS collection from true -> false -> true results in INACTIVE replicas (noble)
+
 ==================  8.11.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStates.java b/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStates.java
index 975f654..c0492f1 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStates.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStates.java
@@ -321,6 +321,10 @@ public class PerReplicaStates implements ReflectMapWriter {
     public int hashCode() {
       return asString.hashCode();
     }
+
+    public State getDuplicate() {
+      return duplicate;
+    }
   }
 
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java b/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java
index 3d3a184..fb3ecc8 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/PerReplicaStatesOps.java
@@ -135,13 +135,24 @@ public class PerReplicaStatesOps {
    * Switch a collection from/to perReplicaState=true
    */
   public static PerReplicaStatesOps modifyCollection(DocCollection coll, boolean enable, PerReplicaStates rs) {
-    return new PerReplicaStatesOps(prs -> enable ? enable(coll) : disable(prs)).init(rs);
+    return new PerReplicaStatesOps(prs -> enable ?
+        enable(coll,prs) :
+        disable(prs)).init(rs);
 
   }
 
-  private static List<PerReplicaStates.Operation> enable(DocCollection coll) {
+  private static List<PerReplicaStates.Operation> enable(DocCollection coll, PerReplicaStates prs) {
     List<PerReplicaStates.Operation> result = new ArrayList<>();
-    coll.forEachReplica((s, r) -> result.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD, new PerReplicaStates.State(r.getName(), r.getState(), r.isLeader(), 0))));
+    coll.forEachReplica((s, r) -> {
+      PerReplicaStates.State st = prs.states.get(r.getName());
+      int newVer = 0;
+      if (st != null) {
+        result.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.DELETE, st));
+        newVer = st.version + 1;
+      }
+      result.add(new PerReplicaStates.Operation(PerReplicaStates.Operation.Type.ADD,
+          new PerReplicaStates.State(r.getName(), r.getState(), r.isLeader(), newVer)));
+    });
     return result;
   }
 
diff --git a/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java b/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
index ad3ce93..988a2d6 100644
--- a/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/cloud/PerReplicaStatesIntegrationTest.java
@@ -20,6 +20,9 @@ package org.apache.solr.common.cloud;
 
 import java.lang.invoke.MethodHandles;
 
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -33,6 +36,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.common.cloud.DocCollection.PER_REPLICA_STATE;
 
 
 /**
@@ -152,4 +156,46 @@ public class PerReplicaStatesIntegrationTest extends SolrCloudTestCase {
     }
 
   }
+
+  public void testMultipleTransitions() throws Exception {
+    String COLL = "prs_modify_op_coll";
+    MiniSolrCloudCluster cluster =
+        configureCluster(2)
+            .withJettyConfig(jetty -> jetty.enableV2(true))
+            .addConfig("conf", getFile("solrj").toPath().resolve("solr").resolve("configsets").resolve("streaming").resolve("conf"))
+            .configure();
+    PerReplicaStates original = null;
+    try {
+      CollectionAdminRequest.createCollection(COLL, "conf", 3, 1)
+          .setPerReplicaState(Boolean.TRUE)
+          .setMaxShardsPerNode(2)
+          .process(cluster.getSolrClient());
+      cluster.waitForActiveCollection(COLL, 3, 3);
+
+      PerReplicaStates prs1 = original =  PerReplicaStates.fetch(ZkStateReader.getCollectionPath(COLL), cluster.getZkClient(), null);
+      log.info("prs1 : {}", prs1);
+
+      CollectionAdminRequest.modifyCollection(COLL,
+          Collections.singletonMap(PER_REPLICA_STATE, "false"))
+          .process(cluster.getSolrClient());
+      cluster.getSolrClient().getZkStateReader().waitForState(COLL, 5, TimeUnit.SECONDS,
+          (liveNodes, collectionState) -> "false".equals(collectionState.getProperties().get(PER_REPLICA_STATE)));
+      CollectionAdminRequest.modifyCollection(COLL,
+          Collections.singletonMap(PER_REPLICA_STATE, "true"))
+          .process(cluster.getSolrClient());
+      cluster.getSolrClient().getZkStateReader().waitForState(COLL, 5, TimeUnit.SECONDS, (liveNodes, collectionState) -> {
+        AtomicBoolean anyFail = new AtomicBoolean(false);
+        PerReplicaStates prs2 = PerReplicaStates.fetch(ZkStateReader.getCollectionPath(COLL), cluster.getZkClient(), null);
+        prs2.states.forEachEntry((r, newState) -> {
+          if(newState.getDuplicate() !=null) anyFail.set(true);
+        });
+        return !anyFail.get();
+      });
+
+    } finally {
+      cluster.shutdown();
+
+    }
+
+  }
 }