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

[lucene-solr] 03/03: @1136 Deal with collections that have EMPTY createNodeSet.

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

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

commit 20adb7783c6e2bbd8ea1a0fd2bfa2a03d3c4810b
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Tue Nov 10 00:13:02 2020 -0600

    @1136 Deal with collections that have EMPTY createNodeSet.
---
 .../cloud/api/collections/CreateCollectionCmd.java | 45 ++++++++++++----------
 .../apache/solr/cloud/ShardRoutingCustomTest.java  | 41 ++++++--------------
 2 files changed, 36 insertions(+), 50 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index 63b8ad7..2cf5a41 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -334,32 +334,35 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
             log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
             //throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core creation failed while creating collection: " + collectionName + "\n" + results);
           } else {
-
+            Object createNodeSet = message.get(ZkStateReader.CREATE_NODE_SET);
+            log.info("createNodeSet={}", createNodeSet);
+            if (createNodeSet == null || (!createNodeSet.equals("") && !createNodeSet.equals(ZkStateReader.CREATE_NODE_SET_EMPTY))) {
             try {
-              zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (l, c) -> {
-                if (c == null) {
-                  return false;
-                }
-                for (String name : coresToCreate.keySet()) {
-                  if (c.getReplica(name) == null || c.getReplica(name).getState() != Replica.State.ACTIVE) {
+                zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (l, c) -> {
+                  if (c == null) {
                     return false;
                   }
-                }
-                Collection<Slice> slices = c.getSlices();
-                if (slices.size() < shardNames.size()) {
-                  return false;
-                }
-                for (Slice slice : slices) {
-                  if (slice.getLeader() == null) {
+                  for (String name : coresToCreate.keySet()) {
+                    if (c.getReplica(name) == null || c.getReplica(name).getState() != Replica.State.ACTIVE) {
+                      return false;
+                    }
+                  }
+                  Collection<Slice> slices = c.getSlices();
+                  if (slices.size() < shardNames.size()) {
                     return false;
                   }
-                }
-                return true;
-              });
-            } catch (InterruptedException e) {
-              log.warn("Interrupted waiting for active replicas on collection creation {}", collectionName);
-            } catch (TimeoutException e) {
-              log.error("Exception waiting for active replicas on collection creation {}", collectionName);
+                  for (Slice slice : slices) {
+                    if (slice.getLeader() == null) {
+                      return false;
+                    }
+                  }
+                  return true;
+                });
+              } catch(InterruptedException e){
+                log.warn("Interrupted waiting for active replicas on collection creation {}", collectionName);
+              } catch(TimeoutException e){
+                log.error("Exception waiting for active replicas on collection creation {}", collectionName);
+              }
             }
 
             if (log.isDebugEnabled()) log.debug("Finished create command on all shards for collection: {}", collectionName);
diff --git a/solr/core/src/test/org/apache/solr/cloud/ShardRoutingCustomTest.java b/solr/core/src/test/org/apache/solr/cloud/ShardRoutingCustomTest.java
index 7bf794c..95273a9 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ShardRoutingCustomTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ShardRoutingCustomTest.java
@@ -22,11 +22,12 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-public class ShardRoutingCustomTest extends AbstractFullDistribZkTestBase {
+public class ShardRoutingCustomTest extends SolrCloudBridgeTestCase {
 
   String collection = DEFAULT_COLLECTION;  // enable this to be configurable (more work needs to be done)
 
@@ -38,50 +39,32 @@ public class ShardRoutingCustomTest extends AbstractFullDistribZkTestBase {
 
   public ShardRoutingCustomTest() {
     schemaString = "schema15.xml";      // we need a string id
+    solrconfigString = "solrconfig.xml";
+    uploadSelectCollection1Config = true;
+    createCollection1 = false;
     sliceCount = 0;
   }
 
   @Test
   public void test() throws Exception {
-    boolean testFinished = false;
-    try {
-      doCustomSharding();
-
-      testFinished = true;
-    } finally {
-      if (!testFinished) {
-        printLayout();
-      }
-    }
+    doCustomSharding();
   }
 
   private void doCustomSharding() throws Exception {
-    printLayout();
 
-    File jettyDir = createTempDir("jetty").toFile();
-    jettyDir.mkdirs();
-    setupJettySolrHome(jettyDir);
-    JettySolrRunner j = createJetty(jettyDir, createTempDir().toFile().getAbsolutePath(), "shardA", "solrconfig.xml", null);
-    j.start();
     assertEquals(0, CollectionAdminRequest
         .createCollection(DEFAULT_COLLECTION, "_default", 1, 1)
-        .setCreateNodeSet("")
+        .setCreateNodeSet(ZkStateReader.CREATE_NODE_SET_EMPTY)
         .process(cloudClient).getStatus());
     assertTrue(CollectionAdminRequest
         .addReplicaToShard(collection,"s1")
-        .setNode(j.getNodeName())
-        .setType(useTlogReplicas()? Replica.Type.TLOG: Replica.Type.NRT)
+        .setNode(cluster.getJettySolrRunner(0).getNodeName())
+        .setType(useTlogReplicas() ? Replica.Type.TLOG: Replica.Type.NRT)
         .process(cloudClient).isSuccess());
-    jettys.add(j);
-    SolrClient client = createNewSolrClient(j.getLocalPort());
-    clients.add(client);
-
-    waitForActiveReplicaCount(cloudClient, DEFAULT_COLLECTION, 1);
-
-    updateMappingsFromZk(this.jettys, this.clients);
-
-    printLayout();
   }
 
+  private boolean useTlogReplicas() {
+    return random().nextBoolean();
+  }
 
 }