You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2020/10/01 23:32:51 UTC

[lucene-solr] branch branch_8_6 updated: SOLR-14663: Copy ConfigSet root data from base ConfigSet when using CREATE command

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

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


The following commit(s) were added to refs/heads/branch_8_6 by this push:
     new e001c22  SOLR-14663: Copy ConfigSet root data from base ConfigSet when using CREATE command
e001c22 is described below

commit e001c2221812a0ba9e9378855040ce72f93eced4
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Thu Oct 1 16:32:18 2020 -0700

    SOLR-14663: Copy ConfigSet root data from base ConfigSet when using CREATE command
---
 solr/CHANGES.txt                                   |   3 +
 .../solr/handler/admin/ConfigSetsHandler.java      |  88 +++-
 .../org/apache/solr/cloud/TestConfigSetsAPI.java   | 513 ++++++++++++---------
 .../solr/cloud/TestConfigSetsAPIExclusivity.java   |   3 +
 .../solr/cloud/TestConfigSetsAPIZkFailure.java     |   2 +
 .../apache/solr/handler/admin/TestConfigsApi.java  |   3 -
 .../apache/solr/common/cloud/ZkConfigManager.java  |  23 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   9 +-
 8 files changed, 396 insertions(+), 248 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d85557e..8c1e487 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -35,6 +35,9 @@ Bug Fixes
 ---------------------
 * SOLR-14751: Zookeeper Admin screen not working for old ZK versions (janhoy)
 
+* SOLR-14663: Copy ConfigSet root data from base ConfigSet when using CREATE command (Andras Salamon, Tomás Fernández Löbbe)
+
+
 ==================  8.6.1 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
index 566a5d8..7418021 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java
@@ -29,7 +29,6 @@ import java.util.zip.ZipInputStream;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.solr.api.Api;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.cloud.OverseerSolrResponse;
@@ -73,6 +72,7 @@ import static org.apache.solr.handler.admin.ConfigSetsHandlerApi.DEFAULT_CONFIGS
  * A {@link org.apache.solr.request.SolrRequestHandler} for ConfigSets API requests.
  */
 public class ConfigSetsHandler extends RequestHandlerBase implements PermissionNameProvider {
+  final public static Boolean DISABLE_CREATE_AUTH_CHECKS = Boolean.getBoolean("solr.disableConfigSetsCreateAuthChecks"); // this is for back compat only
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final CoreContainer coreContainer;
   public static long DEFAULT_ZK_TIMEOUT = 300 * 1000;
@@ -87,19 +87,9 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     this.coreContainer = coreContainer;
   }
 
-
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
-    if (coreContainer == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Core container instance missing");
-    }
-
-    // Make sure that the core is ZKAware
-    if (!coreContainer.isZooKeeperAware()) {
-      throw new SolrException(ErrorCode.BAD_REQUEST,
-          "Solr instance is not running in SolrCloud mode.");
-    }
+    checkErrors();
 
     // Pick the action
     SolrParams params = req.getParams();
@@ -120,6 +110,19 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     rsp.setHttpCaching(false);
   }
 
+  protected void checkErrors() {
+    if (coreContainer == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "Core container instance missing");
+    }
+
+    // Make sure that the core is ZKAware
+    if (!coreContainer.isZooKeeperAware()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "Solr instance is not running in SolrCloud mode.");
+    }
+  }
+
   void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetAction action) throws Exception {
     ConfigSetOperation operation = ConfigSetOperation.get(action);
     if (log.isInfoEnabled()) {
@@ -153,7 +156,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     }
 
     SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
-    String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + Path.SEPARATOR + configSetName;
+    String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSetName;
 
     if (zkClient.exists(configPathInZk, true)) {
       throw new SolrException(ErrorCode.BAD_REQUEST,
@@ -170,7 +173,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     InputStream inputStream = contentStreamsIterator.next().getStream();
 
     // Create a node for the configuration in zookeeper
-    boolean trusted = getTrusted(req);
+    boolean trusted = isTrusted(req, coreContainer.getAuthenticationPlugin());
     zkClient.makePath(configPathInZk, ("{\"trusted\": " + Boolean.toString(trusted) + "}").
         getBytes(StandardCharsets.UTF_8), true);
 
@@ -187,16 +190,30 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     }
     zis.close();
   }
-
-  boolean getTrusted(SolrQueryRequest req) {
-    AuthenticationPlugin authcPlugin = coreContainer.getAuthenticationPlugin();
-    if (log.isInfoEnabled()) {
-      log.info("Trying to upload a configset. authcPlugin: {}, user principal: {}",
-          authcPlugin, req.getUserPrincipal());
+  private static boolean isCurrentlyTrusted(SolrZkClient zkClient, String configSetZkPath) {
+    byte[] configSetNodeContent;
+    try {
+      configSetNodeContent = zkClient.getData(configSetZkPath, null, null, true);
+      if (configSetNodeContent == null || configSetNodeContent.length == 0) {
+        return true;
+      }
+    } catch (KeeperException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Exception while fetching current configSet at " + configSetZkPath, e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while fetching current configSet at " + configSetZkPath, e);
     }
-    if (authcPlugin != null && req.getUserPrincipal() != null) {
+    @SuppressWarnings("unchecked")
+    Map<Object, Object> contentMap = (Map<Object, Object>) Utils.fromJSON(configSetNodeContent);
+    return (boolean) contentMap.getOrDefault("trusted", true);
+  }
+
+  static boolean isTrusted(SolrQueryRequest req, AuthenticationPlugin authPlugin) {
+    if (authPlugin != null && req.getUserPrincipal() != null) {
+      log.debug("Trusted configset request");
       return true;
     }
+    log.debug("Untrusted configset request");
     return false;
   }
 
@@ -271,23 +288,44 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
   enum ConfigSetOperation {
     CREATE_OP(CREATE) {
       @Override
-      Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
+      public Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
         String baseConfigSetName = req.getParams().get(BASE_CONFIGSET, DEFAULT_CONFIGSET_NAME);
+        String newConfigSetName = req.getParams().get(NAME);
+        if (newConfigSetName == null || newConfigSetName.length() == 0) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet name not specified");
+        }
+
+        ZkConfigManager zkConfigManager = new ZkConfigManager(h.coreContainer.getZkController().getZkStateReader().getZkClient());
+        if (zkConfigManager.configExists(newConfigSetName)) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet already exists: " + newConfigSetName);
+        }
+
+        // is there a base config that already exists
+        if (!zkConfigManager.configExists(baseConfigSetName)) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,
+                  "Base ConfigSet does not exist: " + baseConfigSetName);
+        }
+
         Map<String, Object> props = CollectionsHandler.copy(req.getParams().required(), null, NAME);
         props.put(BASE_CONFIGSET, baseConfigSetName);
+        if (!DISABLE_CREATE_AUTH_CHECKS &&
+                !isTrusted(req, h.coreContainer.getAuthenticationPlugin()) &&
+                isCurrentlyTrusted(h.coreContainer.getZkController().getZkClient(), ZkConfigManager.CONFIGS_ZKNODE + "/" +  baseConfigSetName)) {
+          throw new SolrException(ErrorCode.UNAUTHORIZED, "Can't create a configset with an unauthenticated request from a trusted " + BASE_CONFIGSET);
+        }
         return copyPropertiesWithPrefix(req.getParams(), props, PROPERTY_PREFIX + ".");
       }
     },
     DELETE_OP(DELETE) {
       @Override
-      Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
+      public Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
         return CollectionsHandler.copy(req.getParams().required(), null, NAME);
       }
     },
     @SuppressWarnings({"unchecked"})
     LIST_OP(LIST) {
       @Override
-      Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
+      public Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception {
         NamedList<Object> results = new NamedList<>();
         SolrZkClient zk = h.coreContainer.getZkController().getZkStateReader().getZkClient();
         ZkConfigManager zkConfigManager = new ZkConfigManager(zk);
@@ -305,7 +343,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
       this.action = action;
     }
 
-    abstract Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception;
+    public abstract Map<String, Object> call(SolrQueryRequest req, SolrQueryResponse rsp, ConfigSetsHandler h) throws Exception;
 
     public static ConfigSetOperation get(ConfigSetAction action) {
       for (ConfigSetOperation op : values()) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
index 25011be..0d49941 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -16,6 +16,13 @@
  */
 package org.apache.solr.cloud;
 
+import javax.script.ScriptEngineManager;
+import javax.servlet.FilterChain;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
@@ -27,66 +34,67 @@ import java.io.OutputStream;
 import java.lang.invoke.MethodHandles;
 import java.net.URI;
 import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
+import java.security.Principal;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Deque;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipOutputStream;
-import javax.script.ScriptEngineManager;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.commons.io.FileUtils;
 import org.apache.http.HttpEntity;
-import org.apache.http.client.HttpClient;
+import org.apache.http.auth.BasicUserPrincipal;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.entity.ByteArrayEntity;
 import org.apache.http.message.BasicHeader;
 import org.apache.http.util.EntityUtils;
 import org.apache.lucene.util.TestUtil;
-import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
-import org.apache.solr.client.solrj.impl.HttpClientUtil;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Create;
 import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Delete;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkConfigManager;
-import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.ConfigSetParams;
 import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.Base64;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.ConfigSetProperties;
 import org.apache.solr.core.TestDynamicLoading;
-import org.apache.solr.security.BasicAuthIntegrationTest;
+import org.apache.solr.security.AuthenticationPlugin;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.AuthorizationPlugin;
+import org.apache.solr.security.AuthorizationResponse;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.util.ExternalPaths;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
+import org.junit.AfterClass;
+import org.apache.zookeeper.data.Stat;
 import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Ignore;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.noggit.JSONParser;
 import org.slf4j.Logger;
@@ -100,76 +108,114 @@ import static org.junit.matchers.JUnitMatchers.containsString;
 /**
  * Simple ConfigSets API tests on user errors and simple success cases.
  */
-public class TestConfigSetsAPI extends SolrTestCaseJ4 {
+public class TestConfigSetsAPI extends SolrCloudTestCase {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private MiniSolrCloudCluster solrCluster;
 
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-    solrCluster = new MiniSolrCloudCluster(1, createTempDir(), buildJettyConfig("/solr"));
+  private static ZkConfigManager zkConfigManager;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    configureCluster(1)
+            .withSecurityJson(getSecurityJson())
+            .configure();
+    zkConfigManager = new ZkConfigManager(cluster.getZkClient());
+  }
+
+  @AfterClass
+  public static void tearDownClass() throws Exception {
+    zkConfigManager = null;
   }
 
   @Override
   @After
   public void tearDown() throws Exception {
-    if (null != solrCluster) {
-      solrCluster.shutdown();
-      solrCluster = null;
-    }
+    cluster.deleteAllCollections();
+    cluster.deleteAllConfigSets();
     super.tearDown();
   }
 
   @Test
   public void testCreateErrors() throws Exception {
-    final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
-    final SolrClient solrClient = getHttpSolrClient(baseUrl);
-    solrCluster.uploadConfigSet(configset("configset-2"), "configSet");
-
-    // no action
-    CreateNoErrorChecking createNoAction = new CreateNoErrorChecking();
-    createNoAction.setAction(null);
-    verifyException(solrClient, createNoAction, "action");
-
-    // no ConfigSet name
-    CreateNoErrorChecking create = new CreateNoErrorChecking();
-    verifyException(solrClient, create, NAME);
-
-    // set ConfigSet
-    create.setConfigSetName("configSetName");
-
-    // ConfigSet already exists
-    Create alreadyExists = new Create();
-    alreadyExists.setConfigSetName("configSet").setBaseConfigSetName("baseConfigSet");
-    verifyException(solrClient, alreadyExists, "ConfigSet already exists");
-
-    // Base ConfigSet does not exist
-    Create baseConfigNoExists = new Create();
-    baseConfigNoExists.setConfigSetName("newConfigSet").setBaseConfigSetName("baseConfigSet");
-    verifyException(solrClient, baseConfigNoExists, "Base ConfigSet does not exist");
-
-    solrClient.close();
+    final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+    try (final SolrClient solrClient = getHttpSolrClient(baseUrl)) {
+      zkConfigManager.uploadConfigDir(configset("configset-2"), "configSet");
+
+      // no action
+      CreateNoErrorChecking createNoAction = new CreateNoErrorChecking();
+      createNoAction.setAction(null);
+      verifyException(solrClient, createNoAction, "action");
+
+      // no ConfigSet name
+      CreateNoErrorChecking create = new CreateNoErrorChecking();
+      verifyException(solrClient, create, NAME);
+
+      // set ConfigSet
+      create.setConfigSetName("configSetName");
+
+      // ConfigSet already exists
+      Create alreadyExists = new Create();
+      alreadyExists.setConfigSetName("configSet").setBaseConfigSetName("baseConfigSet");
+      verifyException(solrClient, alreadyExists, "ConfigSet already exists");
+
+      // Base ConfigSet does not exist
+      Create baseConfigNoExists = new Create();
+      baseConfigNoExists.setConfigSetName("newConfigSet").setBaseConfigSetName("baseConfigSet");
+      verifyException(solrClient, baseConfigNoExists, "Base ConfigSet does not exist");
+    }
   }
 
   @Test
   public void testCreate() throws Exception {
     // no old, no new
-    verifyCreate(null, "configSet1", null, null);
+    verifyCreate(null, "configSet1", null, null, "solr");
 
     // no old, new
     verifyCreate("baseConfigSet2", "configSet2",
-        null, ImmutableMap.<String, String>of("immutable", "true", "key1", "value1"));
+        null, ImmutableMap.<String, String>of("immutable", "true", "key1", "value1"), "solr");
 
     // old, no new
     verifyCreate("baseConfigSet3", "configSet3",
-        ImmutableMap.<String, String>of("immutable", "false", "key2", "value2"), null);
+        ImmutableMap.<String, String>of("immutable", "false", "key2", "value2"), null, "solr");
 
     // old, new
     verifyCreate("baseConfigSet4", "configSet4",
         ImmutableMap.<String, String>of("immutable", "true", "onlyOld", "onlyOldValue"),
-        ImmutableMap.<String, String>of("immutable", "false", "onlyNew", "onlyNewValue"));
+        ImmutableMap.<String, String>of("immutable", "false", "onlyNew", "onlyNewValue"), "solr");
+  }
+
+  @Test
+  public void testCreateWithTrust() throws Exception {
+    String configsetName = "regular";
+    String configsetSuffix = "testCreateWithTrust";
+    String configsetSuffix2 = "testCreateWithTrust2";
+    uploadConfigSetWithAssertions(configsetName, configsetSuffix, "solr");
+    uploadConfigSetWithAssertions(configsetName, configsetSuffix2, null);
+    try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
+            AbstractZkTestCase.TIMEOUT, 45000, null)) {
+      assertTrue(isTrusted(zkClient, configsetName, configsetSuffix));
+      assertFalse(isTrusted(zkClient, configsetName, configsetSuffix2));
+      try {
+        ignoreException("unauthenticated request");
+        // trusted -> unstrusted
+        createConfigSet(configsetName + configsetSuffix, "foo", Collections.emptyMap(), cluster.getSolrClient(), null);
+        fail("Expecting exception");
+      } catch (SolrException e) {
+        assertEquals(SolrException.ErrorCode.UNAUTHORIZED.code, e.code());
+        unIgnoreException("unauthenticated request");
+      }
+      // trusted -> trusted
+      verifyCreate(configsetName + configsetSuffix, "foo2", Collections.emptyMap(), Collections.emptyMap(), "solr");
+      assertTrue(isTrusted(zkClient, "foo2", ""));
+
+      // unstrusted -> unstrusted
+      verifyCreate(configsetName + configsetSuffix2, "bar", Collections.emptyMap(), Collections.emptyMap(), null);
+      assertFalse(isTrusted(zkClient, "bar", ""));
+
+      // unstrusted -> trusted
+      verifyCreate(configsetName + configsetSuffix2, "bar2", Collections.emptyMap(), Collections.emptyMap(), "solr");
+      assertFalse(isTrusted(zkClient, "bar2", ""));
+    }
   }
 
   private void setupBaseConfigSet(String baseConfigSetName, Map<String, String> oldProps) throws Exception {
@@ -179,39 +225,46 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
     FileUtils.copyDirectory(configDir, tmpConfigDir);
     if (oldProps != null) {
       FileUtils.write(new File(tmpConfigDir, ConfigSetProperties.DEFAULT_FILENAME),
-          getConfigSetProps(oldProps), StandardCharsets.UTF_8);
+          getConfigSetProps(oldProps), UTF_8);
     }
-    solrCluster.uploadConfigSet(tmpConfigDir.toPath(), baseConfigSetName);
+    zkConfigManager.uploadConfigDir(tmpConfigDir.toPath(), baseConfigSetName);
   }
 
   private void verifyCreate(String baseConfigSetName, String configSetName,
-      Map<String, String> oldProps, Map<String, String> newProps) throws Exception {
-    final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
-    final SolrClient solrClient = getHttpSolrClient(baseUrl);
-    setupBaseConfigSet(baseConfigSetName, oldProps);
+      Map<String, String> oldProps, Map<String, String> newProps, String username) throws Exception {
+    final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+    try (final SolrClient solrClient = getHttpSolrClient(baseUrl)) {
+      setupBaseConfigSet(baseConfigSetName, oldProps);
 
-    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
-        AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
-    try {
-      ZkConfigManager configManager = new ZkConfigManager(zkClient);
-      assertFalse(configManager.configExists(configSetName));
-
-      Create create = new Create();
-      create.setBaseConfigSetName(baseConfigSetName).setConfigSetName(configSetName);
-      if (newProps != null) {
-        Properties p = new Properties();
-        p.putAll(newProps);
-        create.setNewConfigSetProperties(p);
+      SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
+              AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
+      try {
+        ZkConfigManager configManager = new ZkConfigManager(zkClient);
+        assertFalse(configManager.configExists(configSetName));
+
+        ConfigSetAdminResponse response = createConfigSet(baseConfigSetName, configSetName, newProps, solrClient, username);
+        assertNotNull(response.getResponse());
+        assertTrue(configManager.configExists(configSetName));
+
+        verifyProperties(configSetName, oldProps, newProps, zkClient);
+      } finally {
+        zkClient.close();
       }
-      ConfigSetAdminResponse response = create.process(solrClient);
-      assertNotNull(response.getResponse());
-      assertTrue(configManager.configExists(configSetName));
+    }
+  }
 
-      verifyProperties(configSetName, oldProps, newProps, zkClient);
-    } finally {
-      zkClient.close();
+  private ConfigSetAdminResponse createConfigSet(String baseConfigSetName, String configSetName, Map<String, String> newProps, SolrClient solrClient, String username) throws SolrServerException, IOException {
+    Create create = new Create();
+    create.setBaseConfigSetName(baseConfigSetName).setConfigSetName(configSetName);
+    if (newProps != null) {
+      Properties p = new Properties();
+      p.putAll(newProps);
+      create.setNewConfigSetProperties(p);
     }
-    solrClient.close();
+    if (username != null) {
+      create.addHeader("user", username);
+    }
+    return create.process(solrClient);
   }
 
   @SuppressWarnings({"rawtypes"})
@@ -225,7 +278,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
     }
 
     if (oldPropsData != null) {
-      InputStreamReader reader = new InputStreamReader(new ByteArrayInputStream(oldPropsData), StandardCharsets.UTF_8);
+      InputStreamReader reader = new InputStreamReader(new ByteArrayInputStream(oldPropsData), UTF_8);
       try {
         return ConfigSetProperties.readFromInputStream(reader);
       } finally {
@@ -281,35 +334,39 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   @Test
   public void testUploadErrors() throws Exception {
-    final SolrClient solrClient = getHttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+    final SolrClient solrClient = getHttpSolrClient(cluster.getJettySolrRunners().get(0).getBaseUrl().toString());
 
     ByteBuffer emptyData = ByteBuffer.allocate(0);
 
+    ignoreException("The configuration name should be provided");
     // Checking error when no configuration name is specified in request
     @SuppressWarnings({"rawtypes"})
-    Map map = postDataAndGetResponse(solrCluster.getSolrClient(),
-        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()
-        + "/admin/configs?action=UPLOAD", emptyData, null, null);
+    Map map = postDataAndGetResponse(cluster.getSolrClient(),
+        cluster.getJettySolrRunners().get(0).getBaseUrl().toString()
+        + "/admin/configs?action=UPLOAD", emptyData, null);
     assertNotNull(map);
+    unIgnoreException("The configuration name should be provided");
     long statusCode = (long) getObjectByPath(map, false,
         Arrays.asList("responseHeader", "status"));
     assertEquals(400l, statusCode);
 
-    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+    SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
         AbstractZkTestCase.TIMEOUT, 45000, null);
 
     // Create dummy config files in zookeeper
     zkClient.makePath("/configs/myconf", true);
     zkClient.create("/configs/myconf/firstDummyFile",
-        "first dummy content".getBytes(StandardCharsets.UTF_8), CreateMode.PERSISTENT, true);
+        "first dummy content".getBytes(UTF_8), CreateMode.PERSISTENT, true);
     zkClient.create("/configs/myconf/anotherDummyFile",
-        "second dummy content".getBytes(StandardCharsets.UTF_8), CreateMode.PERSISTENT, true);
+        "second dummy content".getBytes(UTF_8), CreateMode.PERSISTENT, true);
 
     // Checking error when configuration name specified already exists
-    map = postDataAndGetResponse(solrCluster.getSolrClient(),
-        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()
-        + "/admin/configs?action=UPLOAD&name=myconf", emptyData, null, null);
+    ignoreException("already exists");
+    map = postDataAndGetResponse(cluster.getSolrClient(),
+        cluster.getJettySolrRunners().get(0).getBaseUrl().toString()
+        + "/admin/configs?action=UPLOAD&name=myconf", emptyData, null);
     assertNotNull(map);
+    unIgnoreException("already exists`");
     statusCode = (long) getObjectByPath(map, false,
         Arrays.asList("responseHeader", "status"));
     assertEquals(400l, statusCode);
@@ -324,177 +381,158 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   @Test
   public void testUploadDisabled() throws Exception {
-    try (SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+    try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
         AbstractZkTestCase.TIMEOUT, 45000, null)) {
 
+      ignoreException("Configset upload feature is disabled");
       for (boolean enabled: new boolean[] {true, false}) {
         System.setProperty("configset.upload.enabled", String.valueOf(enabled));
         try {
-          long statusCode = uploadConfigSet("regular", "test-enabled-is-" + enabled, null, null, zkClient);
+          long statusCode = uploadConfigSet("regular", "test-enabled-is-" + enabled, null, zkClient);
           assertEquals("ConfigSet upload enabling/disabling not working as expected for enabled=" + enabled + ".",
               enabled? 0l: 400l, statusCode);
         } finally {
           System.clearProperty("configset.upload.enabled");
         }
       }
+      unIgnoreException("Configset upload feature is disabled");
+    }
+  }
+
+  private boolean isTrusted(SolrZkClient zkClient, String configsetName, String configsetSuffix) throws KeeperException, InterruptedException {
+    String configSetZkPath = String.format(Locale.ROOT,"/configs/%s%s", configsetName, configsetSuffix);
+    byte[] configSetNodeContent = zkClient.getData(configSetZkPath, null, null, true);
+    if (configSetNodeContent == null || configSetNodeContent.length == 0) {
+      return true;
     }
+
+    @SuppressWarnings("unchecked")
+    Map<Object, Object> contentMap = (Map<Object, Object>) Utils.fromJSON(configSetNodeContent);
+    return (boolean) contentMap.getOrDefault("trusted", true);
+  }
+
+  private int getConfigZNodeVersion(SolrZkClient zkClient, String configsetName, String configsetSuffix, String configFile) throws KeeperException, InterruptedException {
+    Stat stat = new Stat();
+    zkClient.getData(String.format(Locale.ROOT, "/configs/%s%s/%s", configsetName, configsetSuffix, configFile), null, stat, true);
+    return stat.getVersion();
   }
 
   @Test
   public void testUpload() throws Exception {
     String suffix = "-untrusted";
-    uploadConfigSetWithAssertions("regular", suffix, null, null);
+    uploadConfigSetWithAssertions("regular", suffix, null);
     // try to create a collection with the uploaded configset
-    createCollection("newcollection", "regular" + suffix, 1, 1, solrCluster.getSolrClient());
+    createCollection("newcollection", "regular" + suffix, 1, 1, cluster.getSolrClient());
   }
-  
+
   @Test
   public void testUploadWithScriptUpdateProcessor() throws Exception {
     Assume.assumeNotNull((new ScriptEngineManager()).getEngineByExtension("js"));
     Assume.assumeNotNull((new ScriptEngineManager()).getEngineByName("JavaScript"));
-    
-      // Authorization off
-      // unprotectConfigsHandler(); // TODO Enable this back when testUploadWithLibDirective() is re-enabled
-      final String untrustedSuffix = "-untrusted";
-      uploadConfigSetWithAssertions("with-script-processor", untrustedSuffix, null, null);
-      // try to create a collection with the uploaded configset
-      Throwable thrown = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
-        createCollection("newcollection2", "with-script-processor" + untrustedSuffix,
-      1, 1, solrCluster.getSolrClient());
-      });
-
+    // Authorization off
+    final String untrustedSuffix = "-untrusted";
+    uploadConfigSetWithAssertions("with-script-processor", untrustedSuffix, null);
+    // try to create a collection with the uploaded configset
+    ignoreException("uploaded without any authentication in place");
+    Throwable thrown = expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> {
+      createCollection("newcollection2", "with-script-processor" + untrustedSuffix,
+              1, 1, cluster.getSolrClient());
+    });
+    unIgnoreException("uploaded without any authentication in place");
     assertThat(thrown.getMessage(), containsString("Underlying core creation failed"));
 
     // Authorization on
     final String trustedSuffix = "-trusted";
-    protectConfigsHandler();
-    uploadConfigSetWithAssertions("with-script-processor", trustedSuffix, "solr", "SolrRocks");
+    uploadConfigSetWithAssertions("with-script-processor", trustedSuffix, "solr");
     // try to create a collection with the uploaded configset
     CollectionAdminResponse resp = createCollection("newcollection2", "with-script-processor" + trustedSuffix,
-    1, 1, solrCluster.getSolrClient());
+            1, 1, cluster.getSolrClient());
     scriptRequest("newcollection2");
 
   }
 
   @Test
-  @Ignore // enable this back when the sleep is removed from protectConfigsHandler() call
   public void testUploadWithLibDirective() throws Exception {
-    // Authorization off
-    unprotectConfigsHandler();
     final String untrustedSuffix = "-untrusted";
-    uploadConfigSetWithAssertions("with-lib-directive", untrustedSuffix, null, null);
+    uploadConfigSetWithAssertions("with-lib-directive", untrustedSuffix, null);
     // try to create a collection with the uploaded configset
-    Throwable thrown = expectThrows(HttpSolrClient.RemoteSolrException.class, () -> {
+    ignoreException("without any authentication in place");
+    Throwable thrown = expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> {
       createCollection("newcollection3", "with-lib-directive" + untrustedSuffix,
-          1, 1, solrCluster.getSolrClient());
+          1, 1, cluster.getSolrClient());
     });
+    unIgnoreException("without any authentication in place");
 
     assertThat(thrown.getMessage(), containsString("Underlying core creation failed"));
 
     // Authorization on
     final String trustedSuffix = "-trusted";
-    protectConfigsHandler();
-    uploadConfigSetWithAssertions("with-lib-directive", trustedSuffix, "solr", "SolrRocks");
+    uploadConfigSetWithAssertions("with-lib-directive", trustedSuffix, "solr");
     // try to create a collection with the uploaded configset
     CollectionAdminResponse resp = createCollection("newcollection3", "with-lib-directive" + trustedSuffix,
-        1, 1, solrCluster.getSolrClient());
+        1, 1, cluster.getSolrClient());
     
     SolrInputDocument doc = sdoc("id", "4055", "subject", "Solr");
-    solrCluster.getSolrClient().add("newcollection3", doc);
-    solrCluster.getSolrClient().commit("newcollection3");
-    assertEquals("4055", solrCluster.getSolrClient().query("newcollection3",
+    cluster.getSolrClient().add("newcollection3", doc);
+    cluster.getSolrClient().commit("newcollection3");
+    assertEquals("4055", cluster.getSolrClient().query("newcollection3",
         params("q", "*:*")).getResults().get(0).get("id"));
   }
 
-  protected SolrZkClient zkClient() {
-    ZkStateReader reader = solrCluster.getSolrClient().getZkStateReader();
-    if (reader == null)
-      solrCluster.getSolrClient().connect();
-    return solrCluster.getSolrClient().getZkStateReader().getZkClient();
-  }
-
-  private void unprotectConfigsHandler() throws Exception {
-    HttpClient cl = null;
-    try {
-      cl = HttpClientUtil.createClient(null);
-      zkClient().setData("/security.json", "{}".getBytes(UTF_8), true);
-    } finally {
-      if (cl != null) {
-        HttpClientUtil.close(cl);
-      }
-    }
-    Thread.sleep(1000); // TODO: Without a delay, the test fails. Some problem with Authc/Authz framework?
-  }
-  
-  private void protectConfigsHandler() throws Exception {
-    String authcPrefix = "/admin/authentication";
-    String authzPrefix = "/admin/authorization";
-
+  private static String getSecurityJson() throws KeeperException, InterruptedException {
     String securityJson = "{\n" +
-        "  'authentication':{\n" +
-        "    'class':'solr.BasicAuthPlugin',\n" +
-        "    'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y= Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" +
-        "  'authorization':{\n" +
-        "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
-        "    'user-role':{'solr':'admin'},\n" +
-        "    'permissions':[{'name':'security-edit','role':'admin'}, {'name':'config-edit','role':'admin'}]}}";
-
-    HttpClient cl = null;
-    try {
-      cl = HttpClientUtil.createClient(null);
-      JettySolrRunner randomJetty = solrCluster.getRandomJetty(random());
-      String baseUrl = randomJetty.getBaseUrl().toString();
-
-      zkClient().setData("/security.json", securityJson.replaceAll("'", "\"").getBytes(UTF_8), true);
-      BasicAuthIntegrationTest.verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/class", "solr.BasicAuthPlugin", 50);
-      BasicAuthIntegrationTest.verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/class", "solr.RuleBasedAuthorizationPlugin", 50);
-    } finally {
-      if (cl != null) {
-        HttpClientUtil.close(cl);
-      }
-    }
-    Thread.sleep(1000); // TODO: Without a delay, the test fails. Some problem with Authc/Authz framework?
+            "  'authentication':{\n" +
+            "    'class':'" + MockAuthenticationPlugin.class.getName() + "'},\n" +
+            "  'authorization':{\n" +
+            "    'class':'" + MockAuthorizationPlugin.class.getName() + "'}}";
+    return securityJson;
   }
 
-  private void uploadConfigSetWithAssertions(String configSetName, String suffix, String username, String password) throws Exception {
-    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+  private void uploadConfigSetWithAssertions(String configSetName, String suffix, String username) throws Exception {
+    SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
         AbstractZkTestCase.TIMEOUT, 45000, null);
     try {
-      long statusCode = uploadConfigSet(configSetName, suffix, username, password, zkClient);
+      long statusCode = uploadConfigSet(configSetName, suffix, username, zkClient);
       assertEquals(0l, statusCode);
-
-      assertTrue("managed-schema file should have been uploaded",
-          zkClient.exists("/configs/"+configSetName+suffix+"/managed-schema", true));
-      assertTrue("managed-schema file contents on zookeeper are not exactly same as that of the file uploaded in config",
-          Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/managed-schema", null, null, true),
-              readFile("solr/configsets/upload/"+configSetName+"/managed-schema")));
-
-      assertTrue("solrconfig.xml file should have been uploaded",
-          zkClient.exists("/configs/"+configSetName+suffix+"/solrconfig.xml", true));
-      byte data[] = zkClient.getData("/configs/"+configSetName+suffix, null, null, true);
-      //assertEquals("{\"trusted\": false}", new String(data, StandardCharsets.UTF_8));
-      assertTrue("solrconfig.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
-          Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/solrconfig.xml", null, null, true),
-              readFile("solr/configsets/upload/"+configSetName+"/solrconfig.xml")));
+      assertConfigsetFiles(configSetName, suffix, zkClient);
     } finally {
       zkClient.close();
     }
   }
+  private void assertConfigsetFiles(String configSetName, String suffix, SolrZkClient zkClient) throws KeeperException, InterruptedException, IOException {
+    assertTrue("managed-schema file should have been uploaded",
+        zkClient.exists("/configs/"+configSetName+suffix+"/managed-schema", true));
+    assertTrue("managed-schema file contents on zookeeper are not exactly same as that of the file uploaded in config",
+        Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/managed-schema", null, null, true),
+            readFile("solr/configsets/upload/"+configSetName+"/managed-schema")));
 
-  private long uploadConfigSet(String configSetName, String suffix, String username, String password,
-      SolrZkClient zkClient) throws IOException {
+    assertTrue("solrconfig.xml file should have been uploaded",
+        zkClient.exists("/configs/"+configSetName+suffix+"/solrconfig.xml", true));
+    byte data[] = zkClient.getData("/configs/"+configSetName+suffix, null, null, true);
+    //assertEquals("{\"trusted\": false}", new String(data, StandardCharsets.UTF_8));
+    assertTrue("solrconfig.xml file contents on zookeeper are not exactly same as that of the file uploaded in config",
+        Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/solrconfig.xml", null, null, true),
+            readFile("solr/configsets/upload/"+configSetName+"/solrconfig.xml")));
+  }
+
+  private long uploadConfigSet(String configSetName, String suffix, String username,
+                               SolrZkClient zkClient) throws IOException {
+    ZkConfigManager configManager = new ZkConfigManager(zkClient);
+    assertFalse(configManager.configExists(configSetName + suffix));
+    return uploadConfigSet(configSetName, suffix, username, zkClient, false, false);
+  }
+
+  private long uploadConfigSet(String configSetName, String suffix, String username,
+      SolrZkClient zkClient, boolean overwrite, boolean cleanup) throws IOException {
     // Read zipped sample config
     ByteBuffer sampleZippedConfig = TestDynamicLoading
         .getFileContent(
             createTempZipFile("solr/configsets/upload/"+configSetName), false);
 
-    ZkConfigManager configManager = new ZkConfigManager(zkClient);
-    assertFalse(configManager.configExists(configSetName+suffix));
-
     @SuppressWarnings({"rawtypes"})
-    Map map = postDataAndGetResponse(solrCluster.getSolrClient(),
-        solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&name="+configSetName+suffix,
-        sampleZippedConfig, username, password);
+    Map map = postDataAndGetResponse(cluster.getSolrClient(),
+        cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&name="+configSetName+suffix + (overwrite? "&overwrite=true" : "") + (cleanup? "&cleanup=true" : ""),
+        sampleZippedConfig, username);
     assertNotNull(map);
     long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
     return statusCode;
@@ -505,7 +543,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
    * and return the path for the zip file.
    */
   private String createTempZipFile(String directoryPath) {
-    File zipFile = new File(solrCluster.getBaseDir().toFile().getAbsolutePath() +
+    File zipFile = new File(cluster.getBaseDir().toFile().getAbsolutePath() +
         File.separator + TestUtil.randomSimpleString(random(), 6, 8) + ".zip");
 
     File directory = TestDynamicLoading.getFile(directoryPath);
@@ -565,7 +603,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   }
   
   public void scriptRequest(String collection) throws SolrServerException, IOException {
-    SolrClient client = solrCluster.getSolrClient();
+    SolrClient client = cluster.getSolrClient();
     SolrInputDocument doc = sdoc("id", "4055", "subject", "Solr");
     client.add(collection, doc);
     client.commit(collection);
@@ -592,7 +630,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   
   @SuppressWarnings({"rawtypes"})
   public static Map postDataAndGetResponse(CloudSolrClient cloudClient,
-      String uri, ByteBuffer bytarr, String username, String password) throws IOException {
+      String uri, ByteBuffer bytarr, String username) throws IOException {
     HttpPost httpPost = null;
     HttpEntity entity;
     String response = null;
@@ -602,19 +640,17 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
       httpPost = new HttpPost(uri);
       
       if (username != null) {
-        String userPass = username + ":" + password;
-        String encoded = Base64.byteArrayToBase64(userPass.getBytes(UTF_8));
-        BasicHeader header = new BasicHeader("Authorization", "Basic " + encoded);
-        httpPost.setHeader(header);
+        httpPost.addHeader(new BasicHeader("user", username));
       }
 
       httpPost.setHeader("Content-Type", "application/octet-stream");
       httpPost.setEntity(new ByteArrayEntity(bytarr.array(), bytarr
           .arrayOffset(), bytarr.limit()));
+      log.info("Uploading configset with user {}", username);
       entity = cloudClient.getLbClient().getHttpClient().execute(httpPost)
           .getEntity();
       try {
-        response = EntityUtils.toString(entity, StandardCharsets.UTF_8);
+        response = EntityUtils.toString(entity, UTF_8);
         m = (Map) Utils.fromJSONString(response);
       } catch (JSONParser.ParseException e) {
         System.err.println("err response: " + response);
@@ -659,7 +695,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   
   @Test
   public void testDeleteErrors() throws Exception {
-    final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+    final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
     final SolrClient solrClient = getHttpSolrClient(baseUrl);
     final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
     final File tmpConfigDir = createTempDir().toFile();
@@ -667,8 +703,8 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
     // Ensure ConfigSet is immutable
     FileUtils.copyDirectory(configDir, tmpConfigDir);
     FileUtils.write(new File(tmpConfigDir, "configsetprops.json"),
-        getConfigSetProps(ImmutableMap.<String, String>of("immutable", "true")), StandardCharsets.UTF_8);
-    solrCluster.uploadConfigSet(tmpConfigDir.toPath(), "configSet");
+        getConfigSetProps(ImmutableMap.<String, String>of("immutable", "true")), UTF_8);
+    zkConfigManager.uploadConfigDir(tmpConfigDir.toPath(), "configSet");
 
     // no ConfigSet name
     DeleteNoErrorChecking delete = new DeleteNoErrorChecking();
@@ -688,19 +724,21 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   private void verifyException(SolrClient solrClient,
                                @SuppressWarnings({"rawtypes"})ConfigSetAdminRequest request,
       String errorContains) throws Exception {
+    ignoreException(errorContains);
     Exception e = expectThrows(Exception.class, () -> solrClient.request(request));
     assertTrue("Expected exception message to contain: " + errorContains
         + " got: " + e.getMessage(), e.getMessage().contains(errorContains));
+    unIgnoreException(errorContains);
   }
 
   @Test
   public void testDelete() throws Exception {
-    final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+    final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
     final SolrClient solrClient = getHttpSolrClient(baseUrl);
-    final String configSet = "configSet";
-    solrCluster.uploadConfigSet(configset("configset-2"), configSet);
+    final String configSet = "testDelete";
+    zkConfigManager.uploadConfigDir(configset("configset-2"), configSet);
 
-    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+    SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
         AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
     try {
       ZkConfigManager configManager = new ZkConfigManager(zkClient);
@@ -720,10 +758,10 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
 
   @Test
   public void testList() throws Exception {
-    final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString();
+    final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
     final SolrClient solrClient = getHttpSolrClient(baseUrl);
 
-    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+    SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
         AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null);
     try {
       // test empty
@@ -736,7 +774,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
       Set<String> configSets = new HashSet<String>();
       for (int i = 0; i < 5; ++i) {
         String configSet = "configSet" + i;
-        solrCluster.uploadConfigSet(configset("configset-2"), configSet);
+        zkConfigManager.uploadConfigDir(configset("configset-2"), configSet);
         configSets.add(configSet);
       }
       response = list.process(solrClient);
@@ -778,7 +816,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
   }
 
   private StringBuilder getConfigSetProps(Map<String, String> map) {
-    return new StringBuilder(new String(Utils.toJSON(map), StandardCharsets.UTF_8));
+    return new StringBuilder(new String(Utils.toJSON(map), UTF_8));
   }
 
   public static class CreateNoErrorChecking extends ConfigSetAdminRequest.Create {
@@ -811,4 +849,55 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 {
       return params;
     }
   }
+
+  public static class MockAuthenticationPlugin extends AuthenticationPlugin {
+
+    @Override
+    public void init(Map<String, Object> pluginConfig) {
+
+    }
+
+    @Override
+    public boolean doAuthenticate(ServletRequest request, ServletResponse response, FilterChain filterChain) throws Exception {
+      if (((HttpServletRequest)request).getHeader("user") != null) {
+        final Principal p = new BasicUserPrincipal("solr");
+        filterChain.doFilter(wrap((HttpServletRequest)request, p, "solr"), response);
+      } else {
+        filterChain.doFilter(request, response);
+      }
+      return true;
+    }
+
+    HttpServletRequest wrap(HttpServletRequest request, Principal principal, String username) {
+      return new HttpServletRequestWrapper(request) {
+        @Override
+        public Principal getUserPrincipal() {
+          return principal;
+        }
+
+        @Override
+        public String getRemoteUser() {
+          return username;
+        }
+      };
+    }
+  }
+
+  public static class MockAuthorizationPlugin implements AuthorizationPlugin {
+
+    @Override
+    public AuthorizationResponse authorize(AuthorizationContext context) {
+      return AuthorizationResponse.OK;
+    }
+
+    @Override
+    public void init(Map<String, Object> initInfo) {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java
index 20a88e0..bb75e7b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java
@@ -17,6 +17,7 @@
 package org.apache.solr.cloud;
 
 import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
@@ -92,6 +93,8 @@ public class TestConfigSetsAPIExclusivity extends SolrTestCaseJ4 {
 
   private void setupBaseConfigSet(String baseConfigSetName) throws Exception {
     solrCluster.uploadConfigSet(configset("configset-2"), baseConfigSetName);
+    //Make configset untrusted
+    solrCluster.getZkClient().setData("/configs/" + baseConfigSetName, "{\"trusted\": false}".getBytes(StandardCharsets.UTF_8), true);
   }
 
   private Exception getFirstExceptionOrNull(List<Exception> list) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
index a15d5a2..b3d7cf2 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
@@ -136,6 +136,8 @@ public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
           getConfigSetProps(oldProps), StandardCharsets.UTF_8);
     }
     solrCluster.uploadConfigSet(tmpConfigDir.toPath(), baseConfigSetName);
+    //Make configset untrusted
+    solrCluster.getZkClient().setData("/configs/" + baseConfigSetName, "{\"trusted\": false}".getBytes(StandardCharsets.UTF_8), true);
   }
 
   private StringBuilder getConfigSetProps(Map<String, String> map) {
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java b/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java
index 0452a1e..643200f 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestConfigsApi.java
@@ -29,7 +29,6 @@ import org.apache.zookeeper.KeeperException;
 
 import static java.util.Collections.EMPTY_MAP;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
 import static org.apache.solr.handler.admin.TestCollectionAPIs.compareOutput;
 
@@ -54,8 +53,6 @@ public class TestConfigsApi extends SolrTestCaseJ4 {
       compareOutput(apiBag, "/cluster/configs/sample", DELETE, null, null,
           "{name :sample, operation:delete}");
 
-      compareOutput(apiBag, "/cluster/configs", POST, "{create:{name : newconf, baseConfigSet: sample }}", null,
-          "{operation:create, name :newconf,  baseConfigSet: sample, immutable: false }");
     }
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java
index aa6404b..72dacf2 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java
@@ -138,11 +138,7 @@ public class ZkConfigManager {
       for (String file : files) {
         List<String> children = zkClient.getChildren(fromZkPath + "/" + file, null, true);
         if (children.size() == 0) {
-          final String toZkFilePath = toZkPath + "/" + file;
-          log.info("Copying zk node {}/{} to {}", fromZkPath, file, toZkFilePath);
-          byte[] data = zkClient.getData(fromZkPath + "/" + file, null, null, true);
-          zkClient.makePath(toZkFilePath, data, true);
-          if (copiedToZkPaths != null) copiedToZkPaths.add(toZkFilePath);
+          copyData(copiedToZkPaths, fromZkPath + "/" + file, toZkPath + "/" + file);
         } else {
           copyConfigDirFromZk(fromZkPath + "/" + file, toZkPath + "/" + file, copiedToZkPaths);
         }
@@ -153,6 +149,13 @@ public class ZkConfigManager {
     }
   }
 
+  private void copyData(Set<String> copiedToZkPaths, String fromZkFilePath, String toZkFilePath) throws KeeperException, InterruptedException {
+    log.info("Copying zk node {} to {}", fromZkFilePath, toZkFilePath);
+    byte[] data = zkClient.getData(fromZkFilePath, null, null, true);
+    zkClient.makePath(toZkFilePath, data, true);
+    if (copiedToZkPaths != null) copiedToZkPaths.add(toZkFilePath);
+  }
+
   /**
    * Copy a config in ZooKeeper
    *
@@ -174,7 +177,15 @@ public class ZkConfigManager {
    * @throws IOException if an I/O error occurs
    */
   public void copyConfigDir(String fromConfig, String toConfig, Set<String> copiedToZkPaths) throws IOException {
-    copyConfigDirFromZk(CONFIGS_ZKNODE + "/" + fromConfig, CONFIGS_ZKNODE + "/" + toConfig, copiedToZkPaths);
+    String fromConfigPath = CONFIGS_ZKNODE + "/" + fromConfig;
+    String toConfigPath = CONFIGS_ZKNODE + "/" + toConfig;
+    try {
+      copyData(copiedToZkPaths, fromConfigPath, toConfigPath);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error config " + fromConfig + " to " + toConfig,
+              SolrZkClient.checkInterrupted(e));
+    }
+    copyConfigDirFromZk(fromConfigPath, toConfigPath, copiedToZkPaths);
   }
 
   // This method is used by configSetUploadTool and CreateTool to resolve the configset directory.
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
index d552e6d..680b027 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
@@ -17,6 +17,7 @@
 package org.apache.solr.cloud;
 
 import java.io.IOException;
+
 import java.lang.invoke.MethodHandles;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
@@ -46,7 +47,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import javax.servlet.Filter;
 
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettyConfig;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.embedded.SSLConfig;
@@ -78,6 +78,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.codahale.metrics.MetricRegistry;
+import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME;
 
 /**
  * "Mini" SolrCloud cluster to be used for testing
@@ -585,7 +586,7 @@ public class MiniSolrCloudCluster {
 
   }
   
-  public void deleteAllConfigSets() throws SolrServerException, IOException {
+  public void deleteAllConfigSets() throws Exception {
 
     List<String> configSetNames = new ConfigSetAdminRequest.List().process(solrClient).getConfigSets();
 
@@ -593,6 +594,10 @@ public class MiniSolrCloudCluster {
       if (configSet.equals("_default")) {
         continue;
       }
+      try {
+        // cleanup any property before removing the configset
+        getZkClient().delete(ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet + "/" + DEFAULT_FILENAME, -1, true);
+      } catch (KeeperException.NoNodeException nne) { }
       new ConfigSetAdminRequest.Delete()
           .setConfigSetName(configSet)
           .process(solrClient);