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 21:58:56 UTC

[lucene-solr] branch branch_8x 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_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


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

commit 83d2ba1ad3a09f4d4788ee527652164f5511f469
Author: Tomas Fernandez Lobbe <tf...@apache.org>
AuthorDate: Thu Oct 1 14:13:40 2020 -0700

    SOLR-14663: Copy ConfigSet root data from base ConfigSet when using CREATE command
---
 solr/CHANGES.txt                                   |   2 +
 .../solr/handler/admin/ConfigSetsHandler.java      |  50 +++++--
 .../org/apache/solr/cloud/TestConfigSetsAPI.java   | 146 +++++++++++++--------
 .../solr/cloud/TestConfigSetsAPIExclusivity.java   |   3 +
 .../solr/cloud/TestConfigSetsAPIZkFailure.java     |   1 +
 .../apache/solr/handler/admin/TestConfigsApi.java  |   4 -
 .../apache/solr/common/cloud/ZkConfigManager.java  |  23 +++-
 7 files changed, 155 insertions(+), 74 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a92e139..f19584a 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -181,6 +181,8 @@ 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 ==================
 
 Bug Fixes
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 ff1a255..6d94fba 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
@@ -73,6 +73,7 @@ import static org.apache.solr.common.params.ConfigSetParams.ConfigSetAction.LIST
  * 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
   final public static String DEFAULT_CONFIGSET_NAME = "_default";
   final public static String AUTOCREATED_CONFIGSET_SUFFIX = ".AUTOCREATED";
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@@ -192,7 +193,7 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     } else {
       filesToDelete = Collections.emptySet();
     }
-    createBaseZnode(zkClient, overwritesExisting, isTrusted(req), cleanup, configPathInZk);
+    createBaseZnode(zkClient, overwritesExisting, isTrusted(req, coreContainer.getAuthenticationPlugin()), cleanup, configPathInZk);
 
     ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8);
     ZipEntry zipEntry = null;
@@ -259,9 +260,19 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
    * Fail if an untrusted request tries to update a trusted ConfigSet
    */
   private void ensureOverwritingUntrustedConfigSet(SolrZkClient zkClient, String configSetZkPath) {
+    boolean isCurrentlyTrusted = isCurrentlyTrusted(zkClient, configSetZkPath);
+    if (isCurrentlyTrusted) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Trying to make an unstrusted ConfigSet update on a trusted configSet");
+    }
+  }
+
+  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) {
@@ -270,21 +281,15 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
     }
     @SuppressWarnings("unchecked")
     Map<Object, Object> contentMap = (Map<Object, Object>) Utils.fromJSON(configSetNodeContent);
-    boolean isCurrentlyTrusted = (boolean) contentMap.getOrDefault("trusted", true);
-    if (isCurrentlyTrusted) {
-      throw new SolrException(ErrorCode.BAD_REQUEST, "Trying to make an unstrusted ConfigSet update on a trusted configSet");
-    }
+    return (boolean) contentMap.getOrDefault("trusted", true);
   }
 
-  boolean isTrusted(SolrQueryRequest req) {
-    AuthenticationPlugin authcPlugin = coreContainer.getAuthenticationPlugin();
-    if (log.isInfoEnabled()) {
-      log.info("Trying to upload a configset. authcPlugin: {}, user principal: {}",
-          authcPlugin, req.getUserPrincipal());
-    }
-    if (authcPlugin != null && req.getUserPrincipal() != null) {
+  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;
   }
 
@@ -361,8 +366,29 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
       @Override
       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 + ".");
       }
     },
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 e6e1caa..6b521d1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java
@@ -37,6 +37,7 @@ import java.nio.ByteBuffer;
 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;
@@ -69,6 +70,7 @@ 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;
@@ -137,51 +139,84 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
   @Test
   public void testCreateErrors() throws Exception {
     final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
-    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");
-
-    solrClient.close();
+    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 {
@@ -197,33 +232,40 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
   }
 
   private void verifyCreate(String baseConfigSetName, String configSetName,
-      Map<String, String> oldProps, Map<String, String> newProps) throws Exception {
+      Map<String, String> oldProps, Map<String, String> newProps, String username) throws Exception {
     final String baseUrl = cluster.getJettySolrRunners().get(0).getBaseUrl().toString();
-    final SolrClient solrClient = getHttpSolrClient(baseUrl);
-    setupBaseConfigSet(baseConfigSetName, oldProps);
+    try (final SolrClient solrClient = getHttpSolrClient(baseUrl)) {
+      setupBaseConfigSet(baseConfigSetName, oldProps);
 
-    SolrZkClient zkClient = new SolrZkClient(cluster.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"})
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 6829501..c3575ac 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPIZkFailure.java
@@ -137,6 +137,7 @@ public class TestConfigSetsAPIZkFailure extends SolrTestCaseJ4 {
           getConfigSetProps(oldProps), StandardCharsets.UTF_8);
     }
     solrCluster.uploadConfigSet(tmpConfigDir.toPath(), baseConfigSetName);
+    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 a77830f..f0ce7ef 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
@@ -27,7 +27,6 @@ import org.apache.solr.handler.ClusterAPI;
 import org.apache.solr.response.SolrQueryResponse;
 
 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;
 
@@ -58,9 +57,6 @@ public class TestConfigsApi extends SolrTestCaseJ4 {
 //      for (Api api : handler.getApis()) apiBag.register(api, EMPTY_MAP);
       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.