You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2019/06/12 00:12:01 UTC

[lucene-solr] branch branch_8x updated: SOLR-12013: collections API CUSTERSTATUS command fails when configset missing

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new cd809ef  SOLR-12013: collections API CUSTERSTATUS command fails when configset missing
cd809ef is described below

commit cd809ef7673c6f28306061e9349d34243715c5e4
Author: erick <er...@gmail.com>
AuthorDate: Tue Jun 11 13:29:23 2019 -0700

    SOLR-12013: collections API CUSTERSTATUS command fails when configset missing
---
 solr/CHANGES.txt                                   |  2 +
 .../apache/solr/cloud/CloudConfigSetService.java   |  8 +++-
 .../cloud/OverseerConfigSetMessageHandler.java     |  9 +++-
 .../solr/cloud/api/collections/AddReplicaCmd.java  |  5 ++-
 .../apache/solr/handler/admin/ClusterStatus.java   |  8 ++--
 .../org/apache/solr/cloud/SolrCLIZkUtilsTest.java  | 39 +++++------------
 .../cloud/api/collections/TestCollectionAPI.java   | 50 +++++++++++++++++++++-
 .../apache/solr/common/cloud/ZkStateReader.java    |  6 +--
 .../solr/cloud/AbstractDistribZkTestBase.java      | 23 ++++++++++
 9 files changed, 109 insertions(+), 41 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b78aa59..321bb8d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -96,6 +96,8 @@ Bug Fixes
 
 * SOLR-13496: Fix distributed grouping related NullPointerException in JSONWriter.writeSolrDocument (Christine Poerschke)
 
+* SOLR-12013: collections API CUSTERSTATUS command fails when configset missing (Erick Erickson)
+
 Other Changes
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
index 9b16d23..29f665c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudConfigSetService.java
@@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
 import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.solr.core.ConfigSetService;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrResourceLoader;
@@ -52,7 +53,12 @@ public class CloudConfigSetService extends ConfigSetService {
       SolrException.log(log, null, e);
     }
 
-    String configName = zkController.getZkStateReader().readConfigName(cd.getCollectionName());
+    String configName;
+    try {
+      configName = zkController.getZkStateReader().readConfigName(cd.getCollectionName());
+    } catch (KeeperException ex) {
+      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "Specified config does not exist in ZooKeeper: " + cd.getCollectionName());
+    }
     return new ZkSolrResourceLoader(cd.getInstanceDir(), configName, parentLoader.getClassLoader(),
         cd.getSubstitutableProperties(), zkController);
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
index 68e4762..945aba5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
@@ -358,7 +358,14 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
     }
 
     for (Map.Entry<String, DocCollection> entry : zkStateReader.getClusterState().getCollectionsMap().entrySet()) {
-      if (configSetName.equals(zkStateReader.readConfigName(entry.getKey())))
+      String configName = null;
+      try {
+        configName = zkStateReader.readConfigName(entry.getKey());
+      } catch (KeeperException ex) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,
+            "Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]");
+      }
+      if (configSetName.equals(configName))
         throw new SolrException(ErrorCode.BAD_REQUEST,
             "Can not delete ConfigSet as it is currently being used by collection [" + entry.getKey() + "]");
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index 08dc12a..49b1e26 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -68,6 +68,7 @@ import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,7 +93,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
   }
 
   List<ZkNodeProps> addReplica(ClusterState clusterState, ZkNodeProps message, NamedList results, Runnable onComplete)
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, KeeperException {
     log.debug("addReplica() : {}", Utils.toJSONString(message));
 
     String extCollectionName = message.getStr(COLLECTION_PROP);
@@ -204,7 +205,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
         .collect(Collectors.toList());
   }
 
-  private ModifiableSolrParams getReplicaParams(ClusterState clusterState, ZkNodeProps message, NamedList results, String collectionName, DocCollection coll, boolean skipCreateReplicaInClusterState, String asyncId, ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException {
+  private ModifiableSolrParams getReplicaParams(ClusterState clusterState, ZkNodeProps message, NamedList results, String collectionName, DocCollection coll, boolean skipCreateReplicaInClusterState, String asyncId, ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
     if (coll.getStr(WITH_COLLECTION) != null) {
       String withCollectionName = coll.getStr(WITH_COLLECTION);
       DocCollection withCollection = clusterState.getCollection(withCollectionName);
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java b/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
index 463c5b6..50fa95a 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ClusterStatus.java
@@ -150,11 +150,9 @@ public class ClusterStatus {
         String configName = zkStateReader.readConfigName(name);
         collectionStatus.put("configName", configName);
         collectionProps.add(name, collectionStatus);
-      } catch (SolrException e) {
-        if (e.getCause() instanceof KeeperException.NoNodeException)  {
-          // skip this collection because the collection's znode has been deleted
-          // which can happen during aggressive collection removal, see SOLR-10720
-        } else throw e;
+      } catch (KeeperException.NoNodeException ex) {
+        // skip this collection because the configset's znode has been deleted
+        // which can happen during aggressive collection removal, see SOLR-10720
       }
     }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java
index 38c73a4..81c5679 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java
@@ -67,7 +67,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
 
     Path configSet = TEST_PATH().resolve("configsets");
     Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf");
-    copyConfigUp(configSet, "cloud-subdirs", "upconfig1");
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "upconfig1", zkAddr);
     // Now do we have that config up on ZK?
     verifyZkLocalPathsMatch(srcPathCheck, "/configs/upconfig1");
 
@@ -112,7 +112,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
     
     Path configSet = TEST_PATH().resolve("configsets");
     Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf");
-    copyConfigUp(configSet, "cloud-subdirs", "downconfig1");
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "downconfig1", zkAddr);
     // Now do we have that config up on ZK?
     verifyZkLocalPathsMatch(srcPathCheck, "/configs/downconfig1");
 
@@ -133,7 +133,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
     Files.createFile(emptyFile);
 
     // Now copy it up and back and insure it's still a file in the new place
-    copyConfigUp(tmp.getParent(), "myconfset", "downconfig2");
+    AbstractDistribZkTestBase.copyConfigUp(tmp.getParent(), "myconfset", "downconfig2", zkAddr);
     Path tmp2 = createTempDir("downConfigNewPlace2");
     downTool = new SolrCLI.ConfigSetDownloadTool();
     args = new String[]{
@@ -158,7 +158,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
     Path configSet = TEST_PATH().resolve("configsets");
     Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf");
 
-    copyConfigUp(configSet, "cloud-subdirs", "cp1");
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "cp1", zkAddr);
 
     // Now copy it somewhere else on ZK.
     String[] args = new String[]{
@@ -456,8 +456,8 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
 
     Path configSet = TEST_PATH().resolve("configsets");
     Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf");
-    
-    copyConfigUp(configSet, "cloud-subdirs", "mv1");
+
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "mv1", zkAddr);
 
     // Now move it somewhere else.
     String[] args = new String[]{
@@ -534,7 +534,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
 
     Path configSet = TEST_PATH().resolve("configsets");
 
-    copyConfigUp(configSet, "cloud-subdirs", "lister");
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "lister", zkAddr);
 
     // Should only find a single level.
     String[] args = new String[]{
@@ -632,9 +632,9 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
     
     Path configSet = TEST_PATH().resolve("configsets");
     Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf");
-    
-    copyConfigUp(configSet, "cloud-subdirs", "rm1");
-    copyConfigUp(configSet, "cloud-subdirs", "rm2");
+
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "rm1", zkAddr);
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "rm2", zkAddr);
 
     // Should fail if recurse not set.
     String[] args = new String[]{
@@ -690,28 +690,11 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase {
         "-zkHost", zkAddr,
     };
 
-    copyConfigUp(configSet, "cloud-subdirs", "rm3");
+    AbstractDistribZkTestBase.copyConfigUp(configSet, "cloud-subdirs", "rm3", zkAddr);
     res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
     assertFalse("Should fail when trying to remove /.", res == 0);
   }
 
-  // We can use this for testing since the goal is to move "some stuff" up to ZK.
-  // The fact that they're in configsets is irrelevant.
-  private void copyConfigUp(Path configSetDir, String srcConfigSet, String dstConfigName) throws Exception {
-    String[] args = new String[]{
-        "-confname", dstConfigName,
-        "-confdir", srcConfigSet,
-        "-zkHost", zkAddr,
-        "-configsetsDir", configSetDir.toAbsolutePath().toString(),
-    };
-
-    SolrCLI.ConfigSetUploadTool tool = new SolrCLI.ConfigSetUploadTool();
-
-    int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
-    assertEquals("Tool should have returned 0 for success, returned: " + res, res, 0);
-
-  }
-
   // Check that all children of fileRoot are children of zkRoot and vice-versa
   private void verifyZkLocalPathsMatch(Path fileRoot, String zkRoot) throws IOException, KeeperException, InterruptedException {
     verifyAllFilesAreZNodes(fileRoot, zkRoot);
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
index 2adc13e..39dd068 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
@@ -25,13 +25,13 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.Lists;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient.RemoteSolrException;
 import org.apache.solr.cloud.ZkTestServer;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
@@ -44,6 +44,7 @@ import org.apache.solr.common.cloud.DocCollection;
 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.ZkConfigManager;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
@@ -100,6 +101,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
     testCollectionCreationShardNameValidation();
     testAliasCreationNameValidation();
     testShardCreationNameValidation();
+    testNoConfigset();
     testModifyCollection(); // deletes replicationFactor property from collections, be careful adding new tests after this one!
   }
 
@@ -203,6 +205,52 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
     }
   }
 
+  // See  SOLR-12013. We should report something back if the configset has mysteriously disappeared.
+  private void testNoConfigset() throws Exception {
+    String configSet = "delete_config";
+
+    final String collection = "deleted_collection";
+    try (CloudSolrClient client = createCloudClient(null)) {
+      copyConfigUp(TEST_PATH().resolve("configsets"), "cloud-minimal", configSet, client.getZkHost());
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("action", CollectionParams.CollectionAction.CREATE.toString());
+      params.set("name", collection);
+      params.set("numShards", "1");
+      params.set("replicationFactor", "1");
+      params.set("collection.configName", configSet);
+      SolrRequest request = new QueryRequest(params);
+      request.setPath("/admin/collections");
+
+      client.request(request);
+
+      waitForCollection(cloudClient.getZkStateReader(), collection, 1);
+      waitForRecoveriesToFinish(collection, false);
+
+      // Now try deleting the configset and doing a clusterstatus.
+      String parent = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet;
+      deleteThemAll(client.getZkStateReader().getZkClient(), parent);
+      client.getZkStateReader().forciblyRefreshAllClusterStateSlow();
+
+      final CollectionAdminRequest.ClusterStatus req = CollectionAdminRequest.getClusterStatus();
+      NamedList<Object> rsp = client.request(req);
+      NamedList<Object> cluster = (NamedList<Object>) rsp.get("cluster");
+      assertNotNull("Cluster state should not be null", cluster);
+      NamedList<Object> collections = (NamedList<Object>) cluster.get("collections");
+      assertNotNull("Collections should not be null in cluster state", collections);
+      assertNotNull("Testing to insure collections are returned", collections.get(COLLECTION_NAME1));
+      assertNull("Should have failed to find: " + collection + " because the configset was delted. ", collections.get(collection));
+    }
+  }
+
+  private void deleteThemAll(SolrZkClient zkClient, String node) throws KeeperException, InterruptedException {
+    List<String> kids = zkClient.getChildren(node, null, true);
+    for (String kid : kids) {
+      deleteThemAll(zkClient, node + "/" + kid);
+    }
+    zkClient.delete(node, -1, true);
+  }
+
   private void assertCountsForRepFactorAndNrtReplicas(CloudSolrClient client, String collectionName) throws Exception {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("action", CollectionParams.CollectionAction.CLUSTERSTATUS.toString());
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index e09f89e..533e518 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -262,7 +262,7 @@ public class ZkStateReader implements SolrCloseable {
    *
    * @param collection to return config set name for
    */
-  public String readConfigName(String collection) {
+  public String readConfigName(String collection) throws KeeperException {
 
     String configName = null;
 
@@ -281,14 +281,14 @@ public class ZkStateReader implements SolrCloseable {
         String configPath = CONFIGS_ZKNODE + "/" + configName;
         if (!zkClient.exists(configPath, true)) {
           log.error("Specified config=[{}] does not exist in ZooKeeper at location=[{}]", configName, configPath);
-          throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "Specified config does not exist in ZooKeeper: " + configName);
+          throw new KeeperException.NoNodeException(configPath);
         } else {
           log.debug("path=[{}] [{}]=[{}] specified config exists in ZooKeeper", configPath, CONFIGNAME_PROP, configName);
         }
       } else {
         throw new ZooKeeperException(ErrorCode.INVALID_STATE, "No config data found at path: " + path);
       }
-    } catch (KeeperException| InterruptedException e) {
+    } catch (InterruptedException e) {
       SolrZkClient.checkInterrupted(e);
       throw new SolrException(ErrorCode.SERVER_ERROR, "Error loading config name for collection " + collection, e);
     }
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index 5ba0ee2..dcd9870 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -37,6 +37,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.core.Diagnostics;
 import org.apache.solr.core.MockDirectoryFactory;
+import org.apache.solr.util.SolrCLI;
 import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
@@ -324,4 +325,26 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
     zkServer = new ZkTestServer(zkServer.getZkDir(), zkServer.getPort());
     zkServer.run(false);
   }
+
+
+  // Copy a configset up from some path on the local  machine to ZK.
+  // Example usage:
+  //
+  // copyConfigUp(TEST_PATH().resolve("configsets"), "cloud-minimal", "configset-name", zk_address);
+
+  static protected void copyConfigUp(Path configSetDir, String srcConfigSet, String dstConfigName, String zkAddr) throws Exception {
+    String[] args = new String[]{
+        "-confname", dstConfigName,
+        "-confdir", srcConfigSet,
+        "-zkHost", zkAddr,
+        "-configsetsDir", configSetDir.toAbsolutePath().toString(),
+    };
+
+    SolrCLI.ConfigSetUploadTool tool = new SolrCLI.ConfigSetUploadTool();
+
+    int res = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), args));
+    assertEquals("Tool should have returned 0 for success, returned: " + res, res, 0);
+
+  }
+
 }