You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gc...@apache.org on 2015/08/27 07:16:09 UTC

svn commit: r1698079 [2/2] - in /lucene/dev/branches/branch_5x/solr: ./ core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/ core/src/java/org/apache/solr/handler/admin/ core/src/test-files/sol...

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/RollingRestartTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/RollingRestartTest.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/RollingRestartTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/RollingRestartTest.java Thu Aug 27 05:16:07 2015
@@ -57,7 +57,7 @@ public class RollingRestartTest extends
 
 
   public void restartWithRolesTest() throws Exception {
-    String leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
+    String leader = OverseerCollectionConfigSetProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
     assertNotNull(leader);
     log.info("Current overseer leader = {}", leader);
 
@@ -93,10 +93,10 @@ public class RollingRestartTest extends
           sawLiveDesignate = true;
           boolean success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, MAX_WAIT_TIME);
           if (!success) {
-            leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
+            leader = OverseerCollectionConfigSetProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
             if (leader == null)
               log.error("NOOVERSEER election queue is :" +
-                  OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(),
+                  OverseerCollectionConfigSetProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(),
                       OverseerElectionContext.PATH + LeaderElector.ELECTION_NODE));
             fail("No overseer designate as leader found after restart #" + (i + 1) + ": " + leader);
           }
@@ -104,10 +104,10 @@ public class RollingRestartTest extends
         assertTrue("Unable to restart (#" + i + "): " + cloudJetty, ChaosMonkey.start(cloudJetty.jetty));
         boolean success = waitUntilOverseerDesignateIsLeader(cloudClient.getZkStateReader().getZkClient(), designates, MAX_WAIT_TIME);
         if (!success) {
-          leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
+          leader = OverseerCollectionConfigSetProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
           if (leader == null)
             log.error("NOOVERSEER election queue is :" +
-                OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(),
+                OverseerCollectionConfigSetProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(),
                     OverseerElectionContext.PATH + LeaderElector.ELECTION_NODE));
           fail("No overseer leader found after restart #" + (i + 1) + ": " + leader);
         }
@@ -120,7 +120,7 @@ public class RollingRestartTest extends
     
     assertTrue("Test may not be working if we never saw a live designate", sawLiveDesignate);
 
-    leader = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
+    leader = OverseerCollectionConfigSetProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
     assertNotNull(leader);
     log.info("Current overseer leader (after restart) = {}", leader);
 
@@ -135,7 +135,7 @@ public class RollingRestartTest extends
     int stableCheckTimeout = 2000;
     String oldleader = null;
     while (System.nanoTime() < timeout && System.nanoTime() < maxTimeout) {
-      String newLeader = OverseerCollectionProcessor.getLeaderNode(testZkClient);
+      String newLeader = OverseerCollectionConfigSetProcessor.getLeaderNode(testZkClient);
       if (newLeader != null && !newLeader.equals(oldleader)) {
         // the leaders have changed, let's move the timeout further
         timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SimpleCollectionCreateDeleteTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SimpleCollectionCreateDeleteTest.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SimpleCollectionCreateDeleteTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SimpleCollectionCreateDeleteTest.java Thu Aug 27 05:16:07 2015
@@ -33,7 +33,7 @@ public class SimpleCollectionCreateDelet
   @Test
   @ShardsFixed(num = 1)
   public void test() throws Exception {
-    String overseerNode = OverseerCollectionProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
+    String overseerNode = OverseerCollectionConfigSetProcessor.getLeaderNode(cloudClient.getZkStateReader().getZkClient());
     String notOverseerNode = null;
     for (CloudJettyRunner cloudJetty : cloudJettys) {
       if (!overseerNode.equals(cloudJetty.nodeName)) {

Added: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java (added)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud;
+
+import com.google.common.collect.ImmutableMap;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+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.response.ConfigSetAdminResponse;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkConfigManager;
+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.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.ConfigSetProperties;
+import org.apache.zookeeper.KeeperException;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME;
+
+/**
+ * Simple ConfigSets API tests on user errors and simple success cases.
+ */
+public class TestConfigSetsAPI extends SolrTestCaseJ4 {
+
+  private MiniSolrCloudCluster solrCluster;
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    final File solrXml = getFile("solr").toPath().resolve("solr.xml").toFile();
+    solrCluster = new MiniSolrCloudCluster(1, createTempDir().toFile(), solrXml, buildJettyConfig("/solr"));
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    solrCluster.shutdown();
+    super.tearDown();
+  }
+
+  @Test
+  public void testCreateErrors() throws Exception {
+    final SolrClient solrClient =
+        new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+    final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
+    solrCluster.uploadConfigDir(configDir, "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);
+
+    // no base ConfigSet name
+    create.setConfigSetName("configSetName");
+    verifyException(solrClient, create, BASE_CONFIGSET);
+
+    // 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();
+  }
+
+  @Test
+  public void testCreate() throws Exception {
+    // no old, no new
+    verifyCreate("baseConfigSet1", "configSet1", null, null);
+
+    // no old, new
+    verifyCreate("baseConfigSet2", "configSet2",
+        null, ImmutableMap.<String, String>of("immutable", "true", "key1", "value1"));
+
+    // old, no new
+    verifyCreate("baseConfigSet3", "configSet3",
+        ImmutableMap.<String, String>of("immutable", "false", "key2", "value2"), null);
+
+    // old, new
+    verifyCreate("baseConfigSet4", "configSet4",
+        ImmutableMap.<String, String>of("immutable", "true", "onlyOld", "onlyOldValue"),
+        ImmutableMap.<String, String>of("immutable", "false", "onlyNew", "onlyNewValue"));
+  }
+
+  private void setupBaseConfigSet(String baseConfigSetName, Map<String, String> oldProps) throws Exception {
+    final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
+    final File tmpConfigDir = createTempDir().toFile();
+    tmpConfigDir.deleteOnExit();
+    FileUtils.copyDirectory(configDir, tmpConfigDir);
+    if (oldProps != null) {
+      FileUtils.write(new File(tmpConfigDir, ConfigSetProperties.DEFAULT_FILENAME),
+          getConfigSetProps(oldProps));
+    }
+    solrCluster.uploadConfigDir(tmpConfigDir, baseConfigSetName);
+  }
+
+  private void verifyCreate(String baseConfigSetName, String configSetName,
+      Map<String, String> oldProps, Map<String, String> newProps) throws Exception {
+    final SolrClient solrClient =
+        new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+    setupBaseConfigSet(baseConfigSetName, oldProps);
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, 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);
+      }
+      ConfigSetAdminResponse response = create.process(solrClient);
+      assertNotNull(response.getResponse());
+      assertTrue(configManager.configExists(configSetName));
+
+      verifyProperties(configSetName, oldProps, newProps, zkClient);
+    } finally {
+      zkClient.close();
+    }
+    solrClient.close();
+  }
+
+  private NamedList getConfigSetPropertiesFromZk(
+      SolrZkClient zkClient, String path) throws Exception {
+    byte [] oldPropsData = null;
+    try {
+      oldPropsData = zkClient.getData(path, null, null, true);
+    } catch (KeeperException.NoNodeException e) {
+      // okay, properties just don't exist
+    }
+
+    if (oldPropsData != null) {
+      InputStreamReader reader = new InputStreamReader(new ByteArrayInputStream(oldPropsData), StandardCharsets.UTF_8);
+      try {
+        return ConfigSetProperties.readFromInputStream(reader);
+      } finally {
+        reader.close();
+      }
+    }
+    return null;
+  }
+
+  private void verifyProperties(String configSetName, Map<String, String> oldProps,
+       Map<String, String> newProps, SolrZkClient zkClient) throws Exception {
+    NamedList properties = getConfigSetPropertiesFromZk(zkClient,
+        ZkConfigManager.CONFIGS_ZKNODE + "/" + configSetName + "/" + DEFAULT_FILENAME);
+    // let's check without merging the maps, since that's what the MessageHandler does
+    // (since we'd probably repeat any bug in the MessageHandler here)
+    if (oldProps == null && newProps == null) {
+      assertNull(properties);
+      return;
+    }
+    assertNotNull(properties);
+
+    // check all oldProps are in props
+    if (oldProps != null) {
+      for (Map.Entry<String, String> entry : oldProps.entrySet()) {
+        assertNotNull(properties.get(entry.getKey()));
+      }
+    }
+    // check all newProps are in props
+    if (newProps != null) {
+      for (Map.Entry<String, String> entry : newProps.entrySet()) {
+        assertNotNull(properties.get(entry.getKey()));
+      }
+    }
+
+    // check the value in properties are correct
+    Iterator<Map.Entry<String, Object>> it = properties.iterator();
+    while (it.hasNext()) {
+      Map.Entry<String, Object> entry = it.next();
+      String newValue = newProps != null ? newProps.get(entry.getKey()) : null;
+      String oldValue = oldProps != null ? oldProps.get(entry.getKey()) : null;
+      if (newValue != null) {
+        assertTrue(newValue.equals(entry.getValue()));
+      } else if (oldValue != null) {
+        assertTrue(oldValue.equals(entry.getValue()));
+      } else {
+        // not in either
+        assert(false);
+      }
+    }
+  }
+
+  @Test
+  public void testDeleteErrors() throws Exception {
+    final SolrClient solrClient =
+        new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+    final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
+    final File tmpConfigDir = createTempDir().toFile();
+    tmpConfigDir.deleteOnExit();
+    // Ensure ConfigSet is immutable
+    FileUtils.copyDirectory(configDir, tmpConfigDir);
+    FileUtils.write(new File(tmpConfigDir, "configsetprops.json"),
+        getConfigSetProps(ImmutableMap.<String, String>of("immutable", "true")));
+    solrCluster.uploadConfigDir(tmpConfigDir, "configSet");
+
+    // no ConfigSet name
+    DeleteNoErrorChecking delete = new DeleteNoErrorChecking();
+    verifyException(solrClient, delete, NAME);
+
+    // ConfigSet doesn't exist
+    delete.setConfigSetName("configSetBogus");
+    verifyException(solrClient, delete, "ConfigSet does not exist");
+
+    // ConfigSet is immutable
+    delete.setConfigSetName("configSet");
+    verifyException(solrClient, delete, "Requested delete of immutable ConfigSet");
+
+    solrClient.close();
+  }
+
+  private void verifyException(SolrClient solrClient, ConfigSetAdminRequest request,
+      String errorContains) throws Exception {
+    try {
+      solrClient.request(request);
+      Assert.fail("Expected exception");
+    } catch (Exception e) {
+      assertTrue("Expected exception message to contain: " + errorContains
+          + " got: " + e.getMessage(), e.getMessage().contains(errorContains));
+    }
+  }
+
+  @Test
+  public void testDelete() throws Exception {
+    final SolrClient solrClient =
+        new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+    final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
+    final String configSet = "configSet";
+    solrCluster.uploadConfigDir(configDir, configSet);
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, null);
+    try {
+      ZkConfigManager configManager = new ZkConfigManager(zkClient);
+      assertTrue(configManager.configExists(configSet));
+
+      Delete delete = new Delete();
+      delete.setConfigSetName(configSet);
+      ConfigSetAdminResponse response = delete.process(solrClient);
+      assertNotNull(response.getResponse());
+      assertFalse(configManager.configExists(configSet));
+    } finally {
+      zkClient.close();
+    }
+
+    solrClient.close();
+  }
+
+  private StringBuilder getConfigSetProps(Map<String, String> map) {
+    return new StringBuilder(new String(Utils.toJSON(map), StandardCharsets.UTF_8));
+  }
+
+  public static class CreateNoErrorChecking extends ConfigSetAdminRequest.Create {
+    public ConfigSetAdminRequest setAction(ConfigSetAction action) {
+       return super.setAction(action);
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      if (action != null) params.set(ConfigSetParams.ACTION, action.toString());
+      if (configSetName != null) params.set(NAME, configSetName);
+      if (baseConfigSetName != null) params.set("baseConfigSet", baseConfigSetName);
+      return params;
+    }
+  }
+
+  public static class DeleteNoErrorChecking extends ConfigSetAdminRequest.Delete {
+    public ConfigSetAdminRequest setAction(ConfigSetAction action) {
+       return super.setAction(action);
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      if (action != null) params.set(ConfigSetParams.ACTION, action.toString());
+      if (configSetName != null) params.set(NAME, configSetName);
+      return params;
+    }
+  }
+}

Added: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java (added)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIExclusivity.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests the exclusivity of the ConfigSets API.
+ * Submits a number of API requests concurrently and checks that
+ * the responses indicate the requests are handled sequentially for
+ * the same ConfigSet and base ConfigSet.
+ */
+public class TestConfigSetsAPIExclusivity extends SolrTestCaseJ4 {
+  private static Logger log = LoggerFactory
+      .getLogger(TestConfigSetsAPIExclusivity.class);
+
+  private MiniSolrCloudCluster solrCluster;
+  private static final String GRANDBASE_CONFIGSET_NAME = "grandBaseConfigSet1";
+  private static final String BASE_CONFIGSET_NAME = "baseConfigSet1";
+  private static final String CONFIGSET_NAME = "configSet1";
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    final File solrXml = getFile("solr").toPath().resolve("solr.xml").toFile();
+    final File testDir = createTempDir().toFile();
+    solrCluster = new MiniSolrCloudCluster(1, testDir,
+        solrXml, buildJettyConfig("/solr"));
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    solrCluster.shutdown();
+    super.tearDown();
+  }
+
+  @Test
+  public void testAPIExclusivity() throws Exception {
+    int trials = 30;
+    setupBaseConfigSet(GRANDBASE_CONFIGSET_NAME);
+    CreateThread createBaseThread =
+        new CreateThread(solrCluster, BASE_CONFIGSET_NAME, GRANDBASE_CONFIGSET_NAME, trials);
+    CreateThread createThread =
+        new CreateThread(solrCluster, CONFIGSET_NAME, BASE_CONFIGSET_NAME, trials);
+    DeleteThread deleteBaseThread = new DeleteThread(solrCluster, BASE_CONFIGSET_NAME, trials);
+    DeleteThread deleteThread = new DeleteThread(solrCluster, CONFIGSET_NAME, trials);
+    List<ConfigSetsAPIThread> threads = Arrays.asList(
+        createBaseThread, createThread, deleteBaseThread, deleteThread);
+
+    for (ConfigSetsAPIThread thread : threads) {
+      thread.start();
+    }
+    for (ConfigSetsAPIThread thread : threads) {
+      thread.join();
+    }
+    List<Exception> exceptions = new LinkedList<Exception>();
+    for (ConfigSetsAPIThread thread : threads) {
+      exceptions.addAll(thread.getUnexpectedExceptions());
+    }
+    assertEquals("Unexpected exception: " + getFirstExceptionOrNull(exceptions),
+        0, exceptions.size());
+  }
+
+  private void setupBaseConfigSet(String baseConfigSetName) throws Exception {
+    final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
+    final File tmpConfigDir = createTempDir().toFile();
+    tmpConfigDir.deleteOnExit();
+    FileUtils.copyDirectory(configDir, tmpConfigDir);
+    solrCluster.uploadConfigDir(tmpConfigDir, baseConfigSetName);
+  }
+
+  private Exception getFirstExceptionOrNull(List<Exception> list) {
+    return list.size() == 0 ? null : list.get(0);
+  }
+
+  private static abstract class ConfigSetsAPIThread extends Thread {
+    private MiniSolrCloudCluster solrCluster;
+    private int trials;
+    private List<Exception> unexpectedExceptions = new LinkedList<Exception>();
+    private List<String> allowedExceptions = Arrays.asList(new String[] {
+        "ConfigSet already exists",
+        "ConfigSet does not exist to delete",
+        "Base ConfigSet does not exist"});
+
+    public ConfigSetsAPIThread(MiniSolrCloudCluster solrCluster, int trials) {
+      this.solrCluster = solrCluster;
+      this.trials = trials;
+    }
+
+    public abstract ConfigSetAdminRequest createRequest();
+
+    public void run() {
+      final SolrClient solrClient =
+          new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+      ConfigSetAdminRequest request = createRequest();
+
+      for (int i = 0; i < trials; ++i) {
+        try {
+          request.process(solrClient);
+        } catch (Exception e) {
+          verifyException(e);
+        }
+      }
+      try {
+        solrClient.close();
+      } catch (Exception e) {
+        log.error("Error closing client", e);
+      }
+    }
+
+    private void verifyException(Exception e) {
+      for (String ex : allowedExceptions) {
+        if (e.getMessage().contains(ex)) {
+          return;
+        }
+      }
+      unexpectedExceptions.add(e);
+    }
+
+    public List<Exception> getUnexpectedExceptions() {
+      return unexpectedExceptions;
+    }
+  }
+
+  private static class CreateThread extends ConfigSetsAPIThread {
+    private String configSet;
+    private String baseConfigSet;
+
+    public CreateThread(MiniSolrCloudCluster solrCluster, String configSet,
+        String baseConfigSet, int trials) {
+      super(solrCluster, trials);
+      this.configSet = configSet;
+      this.baseConfigSet = baseConfigSet;
+    }
+
+    @Override
+    public ConfigSetAdminRequest createRequest() {
+      Create create = new Create();
+      create.setBaseConfigSetName(baseConfigSet).setConfigSetName(configSet);
+      return create;
+    }
+  }
+
+  private static class DeleteThread extends ConfigSetsAPIThread {
+    private String configSet;
+
+    public DeleteThread(MiniSolrCloudCluster solrCluster, String configSet, int trials) {
+      super(solrCluster, trials);
+      this.configSet = configSet;
+    }
+
+    @Override
+    public ConfigSetAdminRequest createRequest() {
+      Delete delete = new Delete();
+      delete.setConfigSetName(configSet);
+      return delete;
+    }
+  }
+}

Added: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java (added)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud;
+
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.jute.InputArchive;
+import org.apache.jute.OutputArchive;
+import org.apache.jute.Record;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
+import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Create;
+import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
+import org.apache.solr.cloud.ZkTestServer;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkConfigManager;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.ConfigSetProperties;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.quorum.Leader.Proposal;
+import org.apache.zookeeper.server.DataNode;
+import org.apache.zookeeper.server.DataTree;
+import org.apache.zookeeper.server.DataTree.ProcessTxnResult;
+import org.apache.zookeeper.server.Request;
+import org.apache.zookeeper.server.ServerCnxn;
+import org.apache.zookeeper.server.ZKDatabase;
+import org.apache.zookeeper.txn.TxnHeader;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.solr.common.cloud.ZkConfigManager.CONFIGS_ZKNODE;
+
+/**
+ * Test the ConfigSets API under ZK failure.  In particular,
+ * if create fails, ensure proper cleanup occurs so we aren't
+ * left with a partially created ConfigSet.
+ */
+public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
+  private MiniSolrCloudCluster solrCluster;
+  private ZkTestServer zkTestServer;
+  private static final String BASE_CONFIGSET_NAME = "baseConfigSet1";
+  private static final String CONFIGSET_NAME = "configSet1";
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    final File solrXml = getFile("solr").toPath().resolve("solr.xml").toFile();
+    final File testDir = createTempDir().toFile();
+    String zkDir = testDir.getAbsolutePath() + File.separator
+      + "zookeeper/server1/data";
+    zkTestServer = new ZkTestServer(zkDir);
+    zkTestServer.run();
+    zkTestServer.setZKDatabase(
+        new FailureDuringCopyZKDatabase(zkTestServer.getZKDatabase(), zkTestServer));
+    solrCluster = new MiniSolrCloudCluster(1, testDir,
+        solrXml, buildJettyConfig("/solr"), zkTestServer);
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    solrCluster.shutdown();
+    zkTestServer.shutdown();
+    super.tearDown();
+  }
+
+  @Test
+  public void testCreateZkFailure() throws Exception {
+    final SolrClient solrClient =
+        new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
+
+    final Map<String, String> oldProps = ImmutableMap.of("immutable", "true");
+    setupBaseConfigSet(BASE_CONFIGSET_NAME, oldProps);
+
+    SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(),
+        AbstractZkTestCase.TIMEOUT, 45000, null);
+    try {
+      ZkConfigManager configManager = new ZkConfigManager(zkClient);
+      assertFalse(configManager.configExists(CONFIGSET_NAME));
+
+      Create create = new Create();
+      create.setBaseConfigSetName(BASE_CONFIGSET_NAME).setConfigSetName(CONFIGSET_NAME);
+      try {
+        ConfigSetAdminResponse response = create.process(solrClient);
+        Assert.fail("Expected solr exception");
+      } catch (RemoteSolrException se) {
+        // partial creation should have been cleaned up
+        assertFalse(configManager.configExists(CONFIGSET_NAME));
+        assertEquals(SolrException.ErrorCode.SERVER_ERROR.code, se.code());
+      }
+    } finally {
+      zkClient.close();
+    }
+
+    solrClient.close();
+  }
+
+  private void setupBaseConfigSet(String baseConfigSetName, Map<String, String> oldProps) throws Exception {
+    final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
+    final File tmpConfigDir = createTempDir().toFile();
+    tmpConfigDir.deleteOnExit();
+    FileUtils.copyDirectory(configDir, tmpConfigDir);
+    if (oldProps != null) {
+      FileUtils.write(new File(tmpConfigDir, ConfigSetProperties.DEFAULT_FILENAME),
+          getConfigSetProps(oldProps));
+    }
+    solrCluster.uploadConfigDir(tmpConfigDir, baseConfigSetName);
+  }
+
+  private StringBuilder getConfigSetProps(Map<String, String> map) {
+    return new StringBuilder(new String(Utils.toJSON(map), StandardCharsets.UTF_8));
+  }
+
+  private static class FailureDuringCopyZKDatabase extends ForwardingZKDatabase {
+    private final ZkTestServer zkTestServer;
+
+    public FailureDuringCopyZKDatabase(ZKDatabase zkdb, ZkTestServer zkTestServer) {
+      super(zkdb);
+      this.zkTestServer = zkTestServer;
+    }
+
+    @Override
+    public byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException {
+      // we know we are doing a copy when we are getting data from the base config set and
+      // the new config set (partially) exists
+      String zkAddress = zkTestServer.getZkAddress();
+      String chroot = zkAddress.substring(zkAddress.lastIndexOf("/"));
+      if (path.startsWith(chroot + CONFIGS_ZKNODE + "/" + BASE_CONFIGSET_NAME)
+          && !path.contains(ConfigSetProperties.DEFAULT_FILENAME)) {
+        List<String> children = null;
+        try {
+          children = getChildren(chroot + CONFIGS_ZKNODE + "/" + CONFIGSET_NAME, null, null);
+        } catch (KeeperException.NoNodeException e) {}
+        if (children != null && children.size() > 0) {
+          throw new RuntimeException("sample zookeeper error");
+        }
+      }
+      return super.getData(path, stat, watcher);
+    }
+  }
+
+  private static class ForwardingZKDatabase extends ZKDatabase {
+    private ZKDatabase zkdb;
+
+    public ForwardingZKDatabase(ZKDatabase zkdb) {
+      super(null);
+      this.zkdb = zkdb;
+    }
+
+    @Override
+    public boolean isInitialized() {
+      return zkdb.isInitialized();
+    }
+
+    @Override
+    public void clear() {
+      zkdb.clear();
+    }
+
+    @Override
+    public DataTree getDataTree() {
+      return zkdb.getDataTree();
+    }
+
+    @Override
+    public long getmaxCommittedLog() {
+      return zkdb.getmaxCommittedLog();
+    }
+
+    @Override
+    public long getminCommittedLog() {
+      return zkdb.getminCommittedLog();
+    }
+
+    @Override
+    public ReentrantReadWriteLock getLogLock() {
+      return zkdb.getLogLock();
+    }
+
+    @Override
+    public synchronized LinkedList<Proposal> getCommittedLog() {
+      return zkdb.getCommittedLog();
+    }
+
+    @Override
+    public long getDataTreeLastProcessedZxid() {
+      return zkdb.getDataTreeLastProcessedZxid();
+    }
+
+    @Override
+    public void setDataTreeInit(boolean b) {
+      zkdb.setDataTreeInit(b);
+    }
+
+    @Override
+    public Collection<Long> getSessions() {
+      return zkdb.getSessions();
+    }
+
+    @Override
+    public ConcurrentHashMap<Long, Integer> getSessionWithTimeOuts() {
+      return zkdb.getSessionWithTimeOuts();
+    }
+
+    @Override
+    public long loadDataBase() throws IOException {
+      return zkdb.loadDataBase();
+    }
+
+    @Override
+    public void addCommittedProposal(Request request) {
+      zkdb.addCommittedProposal(request);
+    }
+
+    @Override
+    public void removeCnxn(ServerCnxn cnxn) {
+      zkdb.removeCnxn(cnxn);
+    }
+
+    @Override
+    public void killSession(long sessionId, long zxid) {
+      zkdb.killSession(sessionId, zxid);
+    }
+
+    @Override
+    public void dumpEphemerals(PrintWriter pwriter) {
+      zkdb.dumpEphemerals(pwriter);
+    }
+
+    @Override
+    public int getNodeCount() {
+      return zkdb.getNodeCount();
+    }
+
+    @Override
+    public HashSet<String> getEphemerals(long sessionId) {
+      return zkdb.getEphemerals(sessionId);
+    }
+
+    @Override
+    public void setlastProcessedZxid(long zxid) {
+      zkdb.setlastProcessedZxid(zxid);
+    }
+
+    @Override
+    public ProcessTxnResult processTxn(TxnHeader hdr, Record txn) {
+      return zkdb.processTxn(hdr, txn);
+    }
+
+    @Override
+    public Stat statNode(String path, ServerCnxn serverCnxn) throws KeeperException.NoNodeException {
+      return zkdb.statNode(path, serverCnxn);
+    }
+
+    @Override
+    public DataNode getNode(String path) {
+      return zkdb.getNode(path);
+    }
+
+    @Override
+    public List<ACL> convertLong(Long aclL) {
+      return zkdb.convertLong(aclL);
+    }
+
+    @Override
+    public byte[] getData(String path, Stat stat, Watcher watcher)
+    throws KeeperException.NoNodeException {
+      return zkdb.getData(path, stat, watcher);
+    }
+
+    @Override
+    public void setWatches(long relativeZxid, List<String> dataWatches,
+            List<String> existWatches, List<String> childWatches, Watcher watcher) {
+      zkdb.setWatches(relativeZxid, dataWatches, existWatches, childWatches, watcher);
+    }
+
+    @Override
+    public List<ACL> getACL(String path, Stat stat) throws NoNodeException {
+      return zkdb.getACL(path, stat);
+    }
+
+    @Override
+    public List<String> getChildren(String path, Stat stat, Watcher watcher)
+    throws KeeperException.NoNodeException {
+      return zkdb.getChildren(path, stat, watcher);
+    }
+
+    @Override
+    public boolean isSpecialPath(String path) {
+      return zkdb.isSpecialPath(path);
+    }
+
+    @Override
+    public int getAclSize() {
+      return zkdb.getAclSize();
+    }
+
+    @Override
+    public boolean truncateLog(long zxid) throws IOException {
+      return zkdb.truncateLog(zxid);
+    }
+
+    @Override
+    public void deserializeSnapshot(InputArchive ia) throws IOException {
+      zkdb.deserializeSnapshot(ia);
+    }
+
+    @Override
+    public void serializeSnapshot(OutputArchive oa) throws IOException,
+    InterruptedException {
+      zkdb.serializeSnapshot(oa);
+    }
+
+    @Override
+    public boolean append(Request si) throws IOException {
+      return zkdb.append(si);
+    }
+
+    @Override
+    public void rollLog() throws IOException {
+      zkdb.rollLog();
+    }
+
+    @Override
+    public void commit() throws IOException {
+      zkdb.commit();
+    }
+
+    @Override
+    public void close() throws IOException {
+      zkdb.close();
+    }
+  }
+}

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestLeaderElectionZkExpiry.java Thu Aug 27 05:16:07 2015
@@ -78,7 +78,7 @@ public class TestLeaderElectionZkExpiry
         boolean found = false;
         while (System.nanoTime() < timeout) {
           try {
-            String leaderNode = OverseerCollectionProcessor.getLeaderNode(zc);
+            String leaderNode = OverseerCollectionConfigSetProcessor.getLeaderNode(zc);
             if (leaderNode != null && !leaderNode.trim().isEmpty()) {
               log.info("Time={} Overseer leader is = {}", System.nanoTime(), leaderNode);
               found = true;

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java Thu Aug 27 05:16:07 2015
@@ -183,7 +183,7 @@ public class TestRebalanceLeaders extend
   List<String> getOverseerSort(String key) {
     List<String> ret = null;
     try {
-      ret = OverseerCollectionProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(),
+      ret = OverseerCollectionConfigSetProcessor.getSortedElectionNodes(cloudClient.getZkStateReader().getZkClient(),
           "/collections/" + COLLECTION_NAME + "/leader_elect/" + key + "/election");
       return ret;
     } catch (KeeperException e) {

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java Thu Aug 27 05:16:07 2015
@@ -24,6 +24,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.handler.admin.InfoHandler;
+import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -308,6 +309,7 @@ public class TestCoreContainer extends S
       " <str name=\"collectionsHandler\">" + CustomCollectionsHandler.class.getName() + "</str>" +
       " <str name=\"infoHandler\">" + CustomInfoHandler.class.getName() + "</str>" +
       " <str name=\"adminHandler\">" + CustomCoreAdminHandler.class.getName() + "</str>" +
+      " <str name=\"configSetsHandler\">" + CustomConfigSetsHandler.class.getName() + "</str>" +
       "</solr>";
 
   public static class CustomCollectionsHandler extends CollectionsHandler {
@@ -327,6 +329,12 @@ public class TestCoreContainer extends S
       super(cc);
     }
   }
+
+  public static class CustomConfigSetsHandler extends ConfigSetsHandler {
+    public CustomConfigSetsHandler(CoreContainer cc) {
+      super(cc);
+    }
+  }
 
   @Test
   public void testCustomHandlers() throws Exception {

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrXml.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrXml.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/core/TestSolrXml.java Thu Aug 27 05:16:07 2015
@@ -72,6 +72,7 @@ public class TestSolrXml extends SolrTes
     assertEquals("core admin handler class", "testAdminHandler", cfg.getCoreAdminHandlerClass());
     assertEquals("collection handler class", "testCollectionsHandler", cfg.getCollectionsHandlerClass());
     assertEquals("info handler class", "testInfoHandler", cfg.getInfoHandlerClass());
+    assertEquals("config set handler class", "testConfigSetsHandler", cfg.getConfigSetsHandlerClass());
     assertEquals("core load threads", 11, cfg.getCoreLoadThreadCount());
     assertThat("core root dir", cfg.getCoreRootDirectory(), containsString("testCoreRootDirectory"));
     assertEquals("distrib conn timeout", 22, cfg.getDistributedConnectionTimeout());

Added: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java (added)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/request/ConfigSetAdminRequest.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Properties;
+
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.response.ConfigSetAdminResponse;
+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.ContentStream;
+import static org.apache.solr.common.params.CommonParams.NAME;
+
+/**
+ * This class is experimental and subject to change.
+ *
+ * @since solr 5.4
+ */
+public abstract class ConfigSetAdminRequest <Q extends ConfigSetAdminRequest<Q>> extends SolrRequest<ConfigSetAdminResponse> {
+
+  protected ConfigSetAction action = null;
+  protected String configSetName = null;
+
+  protected ConfigSetAdminRequest setAction(ConfigSetAction action) {
+    this.action = action;
+    return this;
+  }
+
+  public ConfigSetAdminRequest() {
+    super(METHOD.GET, "/admin/configs");
+  }
+
+  public ConfigSetAdminRequest(String path) {
+    super (METHOD.GET, path);
+  }
+
+  protected abstract Q getThis();
+
+  @Override
+  public SolrParams getParams() {
+    if (action == null) {
+      throw new RuntimeException( "no action specified!" );
+    }
+    if (configSetName == null) {
+      throw new RuntimeException( "no ConfigSet specified!" );
+    }
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(ConfigSetParams.ACTION, action.toString());
+    params.set(NAME, configSetName);
+    return params;
+  }
+
+  @Override
+  public Collection<ContentStream> getContentStreams() throws IOException {
+    return null;
+  }
+
+  @Override
+  protected ConfigSetAdminResponse createResponse(SolrClient client) {
+    return new ConfigSetAdminResponse();
+  }
+
+  public final Q setConfigSetName(String configSetName) {
+    this.configSetName = configSetName;
+    return getThis();
+  }
+
+  public final String getConfigSetName() {
+    return configSetName;
+  }
+
+  // CREATE request
+  public static class Create extends ConfigSetAdminRequest<Create> {
+    protected static String PROPERTY_PREFIX = "configSetProp";
+    protected String baseConfigSetName;
+    protected Properties properties;
+
+    public Create() {
+      action = ConfigSetAction.CREATE;
+    }
+
+    @Override
+    protected Create getThis() {
+      return this;
+    }
+
+    public final Create setBaseConfigSetName(String baseConfigSetName) {
+      this.baseConfigSetName = baseConfigSetName;
+      return getThis();
+    }
+
+    public final String getBaseConfigSetName() {
+      return baseConfigSetName;
+    }
+
+    public final Create setNewConfigSetProperties(Properties properties) {
+      this.properties = properties;
+      return getThis();
+    }
+
+    public final Properties getNewConfigSetProperties() {
+      return properties;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+      if (baseConfigSetName == null) {
+        throw new RuntimeException( "no Base ConfigSet specified!" );
+      }
+      params.set("baseConfigSet", baseConfigSetName);
+      if (properties != null) {
+        for (Map.Entry entry : properties.entrySet()) {
+          params.set(PROPERTY_PREFIX + "." + entry.getKey().toString(),
+              entry.getValue().toString());
+        }
+      }
+      return params;
+    }
+  }
+
+  // DELETE request
+  public static class Delete extends ConfigSetAdminRequest<Delete> {
+    public Delete() {
+      action = ConfigSetAction.DELETE;
+    }
+
+    @Override
+    protected Delete getThis() {
+      return this;
+    }
+  }
+}

Added: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/response/ConfigSetAdminResponse.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/response/ConfigSetAdminResponse.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/response/ConfigSetAdminResponse.java (added)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/client/solrj/response/ConfigSetAdminResponse.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.response;
+
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * No special handling at this time.
+ */
+public class ConfigSetAdminResponse extends SolrResponseBase
+{
+  @SuppressWarnings("unchecked")
+  public NamedList<String> getErrorMessages()
+  {
+     return (NamedList<String>) getResponse().get( "exceptions" );
+  }
+
+}

Modified: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java (original)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkConfigManager.java Thu Aug 27 05:16:07 2015
@@ -29,6 +29,7 @@ import java.nio.file.SimpleFileVisitor;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Class that manages named configs in Zookeeper
@@ -142,4 +143,81 @@ public class ZkConfigManager {
       throw new IOException("Error listing configs", SolrZkClient.checkInterrupted(e));
     }
   }
+
+  /**
+   * Check whether a config exists in Zookeeper
+   *
+   * @param configName the config to check existance on
+   * @return whether the config exists or not
+   * @throws IOException if an I/O error occurs
+   */
+  public Boolean configExists(String configName) throws IOException {
+    try {
+      return zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error checking whether config exists",
+          SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  /**
+   * Delete a config in ZooKeeper
+   *
+   * @param configName the config to delete
+   * @throws IOException if an I/O error occurs
+   */
+  public void deleteConfigDir(String configName) throws IOException {
+    try {
+      zkClient.clean(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error checking whether config exists",
+          SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  private void copyConfigDirFromZk(String fromZkPath, String toZkPath, Set<String> copiedToZkPaths) throws IOException {
+    try {
+      List<String> files = zkClient.getChildren(fromZkPath, null, true);
+      for (String file : files) {
+        List<String> children = zkClient.getChildren(fromZkPath + "/" + file, null, true);
+        if (children.size() == 0) {
+          final String toZkFilePath = toZkPath + "/" + file;
+          logger.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);
+        } else {
+          copyConfigDirFromZk(fromZkPath + "/" + file, toZkPath + "/" + file, copiedToZkPaths);
+        }
+      }
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error copying nodes from zookeeper path " + fromZkPath + " to " + toZkPath,
+          SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  /**
+   * Copy a config in ZooKeeper
+   *
+   * @param fromConfig the config to copy from
+   * @param toConfig the config to copy to
+   * @throws IOException if an I/O error occurs
+   */
+  public void copyConfigDir(String fromConfig, String toConfig) throws IOException {
+    copyConfigDir(CONFIGS_ZKNODE + "/" + fromConfig, CONFIGS_ZKNODE + "/" + toConfig, null);
+  }
+
+  /**
+   * Copy a config in ZooKeeper
+   *
+   * @param fromConfig the config to copy from
+   * @param toConfig the config to copy to
+   * @param copiedToZkPaths should be an empty Set, will be filled in by function
+                            with the paths that were actually copied to.
+   * @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);
+  }
 }

Added: lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java (added)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,51 @@
+package org.apache.solr.common.params;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Locale;
+
+/**
+ * ConfigSets API related parameters and actions.
+ */
+public interface ConfigSetParams
+{
+  public final static String ACTION = "action";
+
+  public enum ConfigSetAction {
+    CREATE,
+    DELETE;
+
+    public static ConfigSetAction get(String p) {
+      if (p != null) {
+        try {
+          return ConfigSetAction.valueOf( p.toUpperCase(Locale.ROOT) );
+        } catch (Exception ex) {}
+      }
+      return null;
+    }
+
+    public boolean isEqual(String s) {
+      if (s == null) return false;
+      return toString().equals(s.toUpperCase(Locale.ROOT));
+    }
+
+    public String toLower() {
+      return toString().toLowerCase(Locale.ROOT);
+    }
+  }
+}

Added: lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java?rev=1698079&view=auto
==============================================================================
--- lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java (added)
+++ lucene/dev/branches/branch_5x/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestConfigSetAdminRequest.java Thu Aug 27 05:16:07 2015
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Basic error checking of ConfigSetAdminRequests.
+ */
+public class TestConfigSetAdminRequest extends SolrTestCaseJ4 {
+
+  @Test
+  public void testNoAction() {
+    ConfigSetAdminRequest request = new MyConfigSetAdminRequest();
+    request.setConfigSetName("name");
+    verifyException(request, "action");
+  }
+
+  @Test
+  public void testCreate() {
+    ConfigSetAdminRequest.Create create = new ConfigSetAdminRequest.Create();
+    verifyException(create, "ConfigSet");
+    create.setConfigSetName("name");
+    verifyException(create, "Base ConfigSet");
+    create.setBaseConfigSetName("baseConfigSet");
+    create.getParams();
+  }
+
+  @Test
+  public void testDelete() {
+    ConfigSetAdminRequest.Delete delete = new ConfigSetAdminRequest.Delete();
+    verifyException(delete, "ConfigSet");
+  }
+
+  private void verifyException(ConfigSetAdminRequest request, String errorContains) {
+    try {
+      request.getParams();
+      Assert.fail("Expected exception");
+    } catch (Exception e) {
+      assertTrue("Expected exception message to contain: " + errorContains,
+          e.getMessage().contains(errorContains));
+    }
+  }
+
+  private static class MyConfigSetAdminRequest extends ConfigSetAdminRequest<MyConfigSetAdminRequest> {
+      public MyConfigSetAdminRequest() {}
+
+      @Override
+      public MyConfigSetAdminRequest getThis() {
+        return this;
+      }
+    };
+}

Modified: lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java (original)
+++ lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java Thu Aug 27 05:16:07 2015
@@ -64,6 +64,7 @@ public class MiniSolrCloudCluster {
   private static Logger log = LoggerFactory.getLogger(MiniSolrCloudCluster.class);
 
   private final ZkTestServer zkServer;
+  private final boolean externalZkServer;
   private final List<JettySolrRunner> jettys = new LinkedList<>();
   private final File testDir;
   private final CloudSolrClient solrClient;
@@ -125,15 +126,34 @@ public class MiniSolrCloudCluster {
    * @throws Exception if there was an error starting the cluster
    */
   public MiniSolrCloudCluster(int numServers, File baseDir, File solrXml, final JettyConfig jettyConfig) throws Exception {
+    this(numServers, baseDir, solrXml, jettyConfig, null);
+  }
+
+  /**
+   * Create a MiniSolrCloudCluster
+   *
+   * @param numServers number of Solr servers to start
+   * @param baseDir base directory that the mini cluster should be run from
+   * @param solrXml solr.xml file to be uploaded to ZooKeeper
+   * @param jettyConfig Jetty configuration
+   * @param zkTestServer ZkTestServer to use.  If null, one will be created
+   *
+   * @throws Exception if there was an error starting the cluster
+   */
+  public MiniSolrCloudCluster(int numServers, File baseDir, File solrXml, final JettyConfig jettyConfig, ZkTestServer zkTestServer) throws Exception {
 
     this.testDir = baseDir;
     this.jettyConfig = jettyConfig;
 
-    String zkDir = testDir.getAbsolutePath() + File.separator
-      + "zookeeper/server1/data";
-    zkServer = new ZkTestServer(zkDir);
-    zkServer.run();
-    
+    this.externalZkServer = zkTestServer != null;
+    if (!externalZkServer) {
+      String zkDir = testDir.getAbsolutePath() + File.separator
+        + "zookeeper/server1/data";
+      zkTestServer = new ZkTestServer(zkDir);
+      zkTestServer.run();
+    }
+    this.zkServer = zkTestServer;
+
     try(SolrZkClient zkClient = new SolrZkClient(zkServer.getZkHost(),
         AbstractZkTestCase.TIMEOUT, 45000, null)) {
       zkClient.makePath("/solr/solr.xml", solrXml, false, true);
@@ -375,7 +395,9 @@ public class MiniSolrCloudCluster {
       executor.shutdown();
       executor.awaitTermination(2, TimeUnit.SECONDS);
       try {
-        zkServer.shutdown();
+        if (!externalZkServer) {
+          zkServer.shutdown();
+        }
       } finally {
         System.clearProperty("zkHost");
       }

Modified: lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java?rev=1698079&r1=1698078&r2=1698079&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java (original)
+++ lucene/dev/branches/branch_5x/solr/test-framework/src/java/org/apache/solr/cloud/ZkTestServer.java Thu Aug 27 05:16:07 2015
@@ -448,6 +448,14 @@ public class ZkTestServer {
     });
   }
 
+  public ZKDatabase getZKDatabase() {
+    return zkServer.zooKeeperServer.getZKDatabase();
+  }
+
+  public void setZKDatabase(ZKDatabase zkDb) {
+    zkServer.zooKeeperServer.setZKDatabase(zkDb);
+  }
+
   public void run() throws InterruptedException {
     log.info("STARTING ZK TEST SERVER");
     // we don't call super.distribSetUp