You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ds...@apache.org on 2023/01/10 05:59:59 UTC

[solr] 02/03: SOLR-15787: FileSystemConfigSetService: implement fully (#146)

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

dsmiley pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git

commit 9ca10b767552a25c78fef48e3d0b2f12f9263dfa
Author: Nazerke Seidan <se...@gmail.com>
AuthorDate: Mon Dec 12 01:16:18 2022 -0500

    SOLR-15787: FileSystemConfigSetService: implement fully (#146)
    
    It could be useful for putting ConfigSets on a shared file system in SolrCloud.  These methods aren't used in standalone mode.
    
    Co-authored-by: Nazerke Seidan <ns...@salesforce.com>
    Co-authored-by: David Smiley <ds...@salesforce.com>
---
 solr/CHANGES.txt                                   |  17 +-
 .../org/apache/solr/cloud/ZkConfigSetService.java  |   2 +-
 .../org/apache/solr/core/ConfigSetService.java     |   4 +-
 .../solr/core/FileSystemConfigSetService.java      | 180 ++++++++++++++++++---
 .../org/apache/solr/core/TestConfigSetService.java | 130 +++++++++++++++
 .../solr/core/TestFileSystemConfigSetService.java  | 115 +++++++++++++
 6 files changed, 420 insertions(+), 28 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 137f3334f66..10663a00ca4 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -71,25 +71,28 @@ Improvements
 
 * SOLR-16438: Support optional split.setPreferredLeaders prop in shard split command. (Bruno Roustant)
 
-* SOLR-10463: Introduce Builder setter for retryExpiryTime on cloud SolrClients.  Deprecated 
+* SOLR-10463: Introduce Builder setter for retryExpiryTime on cloud SolrClients.  Deprecated
   direct setter setRetryExpiryTime on cloud SolrClients. (Eric Pugh)
 
-* SOLR-10461: Introduce Builder setter for aliveCheckInterval on load balanced SolrClients.  Deprecated 
+* SOLR-10461: Introduce Builder setter for aliveCheckInterval on load balanced SolrClients.  Deprecated
   direct setter setAliveCheckInterval on SolrClients. (Eric Pugh, David Smiley, Alex Deparvu)
 
-* SOLR-10462: Introduce Builder setter for pollQueueTime on ConcurrentUpdateHttp2SolrClient.  Deprecated 
+* SOLR-10462: Introduce Builder setter for pollQueueTime on ConcurrentUpdateHttp2SolrClient.  Deprecated
   direct setter setPollQueueTime on ConcurrentUpdateHttp2SolrClient. (Eric Pugh)
 
 * SOLR-10464: Introduce Builder setter for collectionCacheTtl on cloud SolrClients.  Deprecated
   direct setter setCollectionCacheTTL on cloud SolrClients. (Eric Pugh, David Smiley)
-  
+
 * SOLR-10452: Introduce Builder setter withTheseParamNamesInTheUrl for queryParams, renaming them to urlParamNames
-  to clarify they are parameter names, not the values. Deprecated direct setter setQueryParams and addQueryParams 
+  to clarify they are parameter names, not the values. Deprecated direct setter setQueryParams and addQueryParams
   on SolrClients. (Eric Pugh, David Smiley, Alex Deparvu)
-  
+
 * SOLR-10470: Introduce Builder setter for parallelCacheRefreshes on cloud SolrClients.  Deprecated
   direct setter setParallelCacheRefreshes on cloud SolrClients. (Eric Pugh, David Smiley, Alex Deparvu)
-  
+
+* SOLR-15787: FileSystemConfigSetService: implement the abstraction completely.  It could be useful
+  for putting ConfigSets on a shared file system.  (Nazerke Seidan, David Smiley)
+
 Optimizations
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkConfigSetService.java b/solr/core/src/java/org/apache/solr/cloud/ZkConfigSetService.java
index 44ea1a8b78a..b67d4e7cc50 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkConfigSetService.java
@@ -199,7 +199,7 @@ public class ZkConfigSetService extends ConfigSetService {
       throws IOException {
     String filePath = CONFIGS_ZKNODE + "/" + configName + "/" + fileName;
     try {
-      // if createNew is true then zkClient#makePath failOnExists is set to false
+      // if overwriteOnExists is true then zkClient#makePath failOnExists is set to false
       zkClient.makePath(filePath, data, CreateMode.PERSISTENT, null, !overwriteOnExists, true);
     } catch (KeeperException.NodeExistsException nodeExistsException) {
       throw new SolrException(
diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
index 15896926dba..5d9fbf30054 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -395,8 +395,8 @@ public abstract class ConfigSetService {
   public abstract void uploadConfig(String configName, Path dir) throws IOException;
 
   /**
-   * Upload a file to config If file does not exist, it will be uploaded If createNew param is set
-   * to true then file be overwritten
+   * Upload a file to config If file does not exist, it will be uploaded If overwriteOnExists is set
+   * to true then file will be overwritten
    *
    * @param configName the name to give the config
    * @param fileName the name of the file
diff --git a/solr/core/src/java/org/apache/solr/core/FileSystemConfigSetService.java b/solr/core/src/java/org/apache/solr/core/FileSystemConfigSetService.java
index d8ee6325779..01dcdf8f524 100644
--- a/solr/core/src/java/org/apache/solr/core/FileSystemConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/FileSystemConfigSetService.java
@@ -16,21 +16,31 @@
  */
 package org.apache.solr.core;
 
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.nio.file.FileVisitResult;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Solr Standalone File System ConfigSetService impl.
+ * FileSystem ConfigSetService impl.
  *
  * <p>Loads a ConfigSet defined by the core's configSet property, looking for a directory named for
  * the configSet property value underneath a base directory. If no configSet property is set, loads
@@ -38,6 +48,9 @@ import org.slf4j.LoggerFactory;
  */
 public class FileSystemConfigSetService extends ConfigSetService {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  /** .metadata.json hidden file where metadata is stored */
+  public static final String METADATA_FILE = ".metadata.json";
+
   private final Path configSetBase;
 
   public FileSystemConfigSetService(CoreContainer cc) {
@@ -45,6 +58,12 @@ public class FileSystemConfigSetService extends ConfigSetService {
     this.configSetBase = cc.getConfig().getConfigSetBaseDirectory();
   }
 
+  /** Testing purpose */
+  protected FileSystemConfigSetService(Path configSetBase) {
+    super(null, false);
+    this.configSetBase = configSetBase;
+  }
+
   @Override
   public SolrResourceLoader createCoreResourceLoader(CoreDescriptor cd) {
     Path instanceDir = locateInstanceDir(cd);
@@ -60,68 +79,184 @@ public class FileSystemConfigSetService extends ConfigSetService {
 
   @Override
   public boolean checkConfigExists(String configName) throws IOException {
-    Path solrConfigXmlFile = configSetBase.resolve(configName).resolve("solrconfig.xml");
+    Path solrConfigXmlFile = getConfigDir(configName).resolve("solrconfig.xml");
     return Files.exists(solrConfigXmlFile);
   }
 
   @Override
   public void deleteConfig(String configName) throws IOException {
-    throw new UnsupportedOperationException();
+    deleteDir(getConfigDir(configName));
   }
 
   @Override
   public void deleteFilesFromConfig(String configName, List<String> filesToDelete)
       throws IOException {
-    throw new UnsupportedOperationException();
+    Path configDir = getConfigDir(configName);
+    Objects.requireNonNull(filesToDelete);
+    for (String fileName : filesToDelete) {
+      Path file = configDir.resolve(fileName);
+      if (Files.exists(file)) {
+        if (Files.isDirectory(file)) {
+          deleteDir(file);
+        } else {
+          Files.delete(file);
+        }
+      }
+    }
   }
 
   @Override
   public void copyConfig(String fromConfig, String toConfig) throws IOException {
-    throw new UnsupportedOperationException();
+    Path source = getConfigDir(fromConfig);
+    Path dest = getConfigDir(toConfig);
+    copyRecursively(source, dest);
+  }
+
+  private void deleteDir(Path dir) throws IOException {
+    try {
+      Files.walkFileTree(
+          dir,
+          new SimpleFileVisitor<Path>() {
+            @Override
+            public FileVisitResult visitFile(Path path, BasicFileAttributes attrs)
+                throws IOException {
+              Files.delete(path);
+              return FileVisitResult.CONTINUE;
+            }
+
+            @Override
+            public FileVisitResult postVisitDirectory(Path dir, IOException ioException)
+                throws IOException {
+              Files.delete(dir);
+              return FileVisitResult.CONTINUE;
+            }
+          });
+    } catch (NoSuchFileException e) {
+      // do nothing
+    }
   }
 
   @Override
-  public void uploadConfig(String configName, Path dir) throws IOException {
-    throw new UnsupportedOperationException();
+  public void uploadConfig(String configName, Path source) throws IOException {
+    Path dest = getConfigDir(configName);
+    copyRecursively(source, dest);
   }
 
   @Override
   public void uploadFileToConfig(
       String configName, String fileName, byte[] data, boolean overwriteOnExists)
       throws IOException {
-    throw new UnsupportedOperationException();
+    Path filePath = getConfigDir(configName).resolve(fileName);
+    if (!Files.exists(filePath) || overwriteOnExists) {
+      Files.write(filePath, data);
+    }
   }
 
   @Override
   public void setConfigMetadata(String configName, Map<String, Object> data) throws IOException {
-    throw new UnsupportedOperationException();
+    // store metadata in .metadata.json file
+    Path metadataPath = getConfigDir(configName).resolve(METADATA_FILE);
+    Files.write(metadataPath, Utils.toJSON(data));
   }
 
   @Override
   public Map<String, Object> getConfigMetadata(String configName) throws IOException {
-    throw new UnsupportedOperationException();
+    // get metadata from .metadata.json file
+    Path metadataPath = getConfigDir(configName).resolve(METADATA_FILE);
+    byte[] data = null;
+    try {
+      data = Files.readAllBytes(metadataPath);
+    } catch (NoSuchFileException e) {
+      return Collections.emptyMap();
+    }
+    @SuppressWarnings("unchecked")
+    Map<String, Object> metadata = (Map<String, Object>) Utils.fromJSON(data);
+    return metadata;
   }
 
   @Override
-  public void downloadConfig(String configName, Path dir) throws IOException {
-    throw new UnsupportedOperationException();
+  public void downloadConfig(String configName, Path dest) throws IOException {
+    Path source = getConfigDir(configName);
+    copyRecursively(source, dest);
+  }
+
+  private void copyRecursively(Path source, Path target) throws IOException {
+    try {
+      Files.walkFileTree(
+          source,
+          new SimpleFileVisitor<Path>() {
+            @Override
+            public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs)
+                throws IOException {
+              Files.createDirectories(target.resolve(source.relativize(dir).toString()));
+              return FileVisitResult.CONTINUE;
+            }
+
+            @Override
+            public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+                throws IOException {
+              Files.copy(
+                  file, target.resolve(source.relativize(file).toString()), REPLACE_EXISTING);
+              return FileVisitResult.CONTINUE;
+            }
+          });
+    } catch (NoSuchFileException e) {
+      // do nothing
+    }
   }
 
   @Override
   public List<String> listConfigs() throws IOException {
     try (Stream<Path> configs = Files.list(configSetBase)) {
-      return configs.map(Path::getFileName).map(Path::toString).collect(Collectors.toList());
+      return configs
+          .map(Path::getFileName)
+          .map(Path::toString)
+          .sorted()
+          .collect(Collectors.toList());
     }
   }
 
   @Override
-  public byte[] downloadFileFromConfig(String configName, String filePath) throws IOException {
-    throw new UnsupportedOperationException();
+  public byte[] downloadFileFromConfig(String configName, String fileName) throws IOException {
+    Path filePath = getConfigDir(configName).resolve(fileName);
+    byte[] data = null;
+    try {
+      data = Files.readAllBytes(filePath);
+    } catch (NoSuchFileException e) {
+      // do nothing
+    }
+    return data;
   }
 
   @Override
   public List<String> getAllConfigFiles(String configName) throws IOException {
-    throw new UnsupportedOperationException();
+    Path configDir = getConfigDir(configName);
+    List<String> filePaths = new ArrayList<>();
+    Files.walkFileTree(
+        configDir,
+        new SimpleFileVisitor<Path>() {
+          @Override
+          public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+              throws IOException {
+            // don't include hidden (.) files
+            if (!Files.isHidden(file)) {
+              filePaths.add(configDir.relativize(file).toString());
+              return FileVisitResult.CONTINUE;
+            }
+            return FileVisitResult.CONTINUE;
+          }
+
+          @Override
+          public FileVisitResult postVisitDirectory(Path dir, IOException ioException) {
+            String relativePath = configDir.relativize(dir).toString();
+            if (!relativePath.isEmpty()) {
+              filePaths.add(relativePath + "/");
+            }
+            return FileVisitResult.CONTINUE;
+          }
+        });
+    Collections.sort(filePaths);
+    return filePaths;
   }
 
   protected Path locateInstanceDir(CoreDescriptor cd) {
@@ -136,8 +271,8 @@ public class FileSystemConfigSetService extends ConfigSetService {
   }
 
   @Override
-  protected Long getCurrentSchemaModificationVersion(
-      String configSet, SolrConfig solrConfig, String schemaFileName) throws IOException {
+  public Long getCurrentSchemaModificationVersion(
+      String configSet, SolrConfig solrConfig, String schemaFileName) {
     Path schemaFile = solrConfig.getResourceLoader().getConfigPath().resolve(schemaFileName);
     try {
       return Files.getLastModifiedTime(schemaFile).toMillis();
@@ -148,4 +283,13 @@ public class FileSystemConfigSetService extends ConfigSetService {
       return null; // debatable; we'll see an error soon if there's a real problem
     }
   }
+
+  protected Path getConfigDir(String configName) throws IOException {
+    // startsWith works simply; we must normalize()
+    Path path = configSetBase.resolve(configName).normalize();
+    if (!path.startsWith(configSetBase)) {
+      throw new IOException("configName=" + configName + " is not found under configSetBase dir");
+    }
+    return path;
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java b/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java
new file mode 100644
index 00000000000..07cc09b91a2
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java
@@ -0,0 +1,130 @@
+/*
+ * 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.core;
+
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.ZkConfigSetService;
+import org.apache.solr.cloud.ZkTestServer;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestConfigSetService extends SolrTestCaseJ4 {
+
+  private final ConfigSetService configSetService;
+  private static ZkTestServer zkServer;
+  private static SolrZkClient zkClient;
+
+  @BeforeClass
+  public static void startZkServer() throws Exception {
+    zkServer = new ZkTestServer(createTempDir("zkData"));
+    zkServer.run();
+    zkClient = new SolrZkClient(zkServer.getZkAddress("/solr"), 10000);
+  }
+
+  @AfterClass
+  public static void shutdownZkServer() throws IOException, InterruptedException {
+    zkClient.close();
+    if (null != zkServer) {
+      zkServer.shutdown();
+    }
+    zkServer = null;
+  }
+
+  public TestConfigSetService(Supplier<ConfigSetService> configSetService) {
+    this.configSetService = configSetService.get();
+  }
+
+  @ParametersFactory
+  @SuppressWarnings("rawtypes")
+  public static Iterable<Supplier[]> parameters() {
+    return Arrays.asList(
+        new Supplier[][] {
+          {() -> new ZkConfigSetService(zkClient)},
+          {() -> new FileSystemConfigSetService(createTempDir("configsets"))}
+        });
+  }
+
+  @Test
+  public void testConfigSetServiceOperations() throws IOException {
+    final String configName = "testconfig";
+    byte[] testdata = "test data".getBytes(StandardCharsets.UTF_8);
+
+    Path configDir = createTempDir("testconfig");
+    Files.createFile(configDir.resolve("solrconfig.xml"));
+    Files.write(configDir.resolve("file1"), testdata);
+    Files.createFile(configDir.resolve("file2"));
+    Files.createDirectory(configDir.resolve("subdir"));
+    Files.createFile(configDir.resolve("subdir").resolve("file3"));
+
+    configSetService.uploadConfig(configName, configDir);
+
+    assertTrue(configSetService.checkConfigExists(configName));
+    assertFalse(configSetService.checkConfigExists("dummyConfig"));
+
+    byte[] data = "file3 data".getBytes(StandardCharsets.UTF_8);
+    configSetService.uploadFileToConfig(configName, "subdir/file3", data, true);
+    assertArrayEquals(configSetService.downloadFileFromConfig(configName, "subdir/file3"), data);
+
+    data = "file4 data".getBytes(StandardCharsets.UTF_8);
+    configSetService.uploadFileToConfig(configName, "subdir/file4", data, true);
+    assertArrayEquals(configSetService.downloadFileFromConfig(configName, "subdir/file4"), data);
+
+    Map<String, Object> metadata = configSetService.getConfigMetadata(configName);
+    assertTrue(metadata.isEmpty());
+
+    configSetService.setConfigMetadata(configName, Collections.singletonMap("trusted", true));
+    metadata = configSetService.getConfigMetadata(configName);
+    assertTrue(metadata.containsKey("trusted"));
+
+    configSetService.setConfigMetadata(configName, Collections.singletonMap("foo", true));
+    assertFalse(configSetService.getConfigMetadata(configName).containsKey("trusted"));
+    assertTrue(configSetService.getConfigMetadata(configName).containsKey("foo"));
+
+    List<String> configFiles = configSetService.getAllConfigFiles(configName);
+    assertEquals(
+        configFiles.toString(),
+        "[file1, file2, solrconfig.xml, subdir/, subdir/file3, subdir/file4]");
+
+    List<String> configs = configSetService.listConfigs();
+    assertEquals(configs.toString(), "[testconfig]");
+
+    configSetService.copyConfig(configName, "testconfig.AUTOCREATED");
+    List<String> copiedConfigFiles = configSetService.getAllConfigFiles("testconfig.AUTOCREATED");
+    assertEquals(configFiles.toString(), (copiedConfigFiles.toString()));
+
+    assertEquals(2, configSetService.listConfigs().size());
+
+    configSetService.deleteConfig("testconfig.AUTOCREATED");
+    assertFalse(configSetService.checkConfigExists("testconfig.AUTOCREATED"));
+
+    configSetService.deleteConfig(configName);
+    assertFalse(configSetService.checkConfigExists(configName));
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/core/TestFileSystemConfigSetService.java b/solr/core/src/test/org/apache/solr/core/TestFileSystemConfigSetService.java
new file mode 100644
index 00000000000..7b0e26d47cc
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestFileSystemConfigSetService.java
@@ -0,0 +1,115 @@
+/*
+ * 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.core;
+
+import static org.apache.solr.core.FileSystemConfigSetService.METADATA_FILE;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.commons.io.file.PathUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFileSystemConfigSetService extends SolrTestCaseJ4 {
+  private static Path configSetBase;
+  private static FileSystemConfigSetService fileSystemConfigSetService;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    configSetBase = createTempDir();
+    fileSystemConfigSetService = new FileSystemConfigSetService(configSetBase);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    PathUtils.deleteDirectory(configSetBase);
+    fileSystemConfigSetService = null;
+  }
+
+  @Test
+  public void testUploadAndDeleteConfig() throws IOException {
+    String configName = "testconfig";
+
+    fileSystemConfigSetService.uploadConfig(configName, configset("cloud-minimal"));
+
+    assertEquals(fileSystemConfigSetService.listConfigs().size(), 1);
+    assertTrue(fileSystemConfigSetService.checkConfigExists(configName));
+
+    byte[] testdata = "test data".getBytes(StandardCharsets.UTF_8);
+    fileSystemConfigSetService.uploadFileToConfig(configName, "testfile", testdata, true);
+
+    // metadata is stored in .metadata.json
+    fileSystemConfigSetService.setConfigMetadata(configName, Map.of("key1", "val1"));
+    Map<String, Object> metadata = fileSystemConfigSetService.getConfigMetadata(configName);
+    assertEquals(metadata.toString(), "{key1=val1}");
+
+    List<String> allConfigFiles = fileSystemConfigSetService.getAllConfigFiles(configName);
+    assertEquals(allConfigFiles.toString(), "[schema.xml, solrconfig.xml, testfile]");
+
+    fileSystemConfigSetService.deleteFilesFromConfig(
+        configName, List.of(METADATA_FILE, "testfile"));
+    metadata = fileSystemConfigSetService.getConfigMetadata(configName);
+    assertTrue(metadata.isEmpty());
+
+    allConfigFiles = fileSystemConfigSetService.getAllConfigFiles(configName);
+    assertEquals(allConfigFiles.toString(), "[schema.xml, solrconfig.xml]");
+
+    fileSystemConfigSetService.copyConfig(configName, "copytestconfig");
+    assertEquals(fileSystemConfigSetService.listConfigs().size(), 2);
+
+    allConfigFiles = fileSystemConfigSetService.getAllConfigFiles("copytestconfig");
+    assertEquals(allConfigFiles.toString(), "[schema.xml, solrconfig.xml]");
+
+    Path downloadConfig = createTempDir("downloadConfig");
+    fileSystemConfigSetService.downloadConfig(configName, downloadConfig);
+
+    List<String> configs = getFileList(downloadConfig);
+    assertEquals(configs.toString(), "[schema.xml, solrconfig.xml]");
+
+    Exception ex =
+        assertThrows(
+            IOException.class,
+            () -> {
+              fileSystemConfigSetService.uploadConfig("../dummy", createTempDir("tmp"));
+            });
+    assertTrue(ex.getMessage().startsWith("configName=../dummy is not found under configSetBase"));
+
+    fileSystemConfigSetService.deleteConfig(configName);
+    fileSystemConfigSetService.deleteConfig("copytestconfig");
+
+    assertFalse(fileSystemConfigSetService.checkConfigExists(configName));
+    assertFalse(fileSystemConfigSetService.checkConfigExists("copytestconfig"));
+  }
+
+  private static List<String> getFileList(Path confDir) throws IOException {
+    try (Stream<Path> configs = Files.list(confDir)) {
+      return configs
+          .map(Path::getFileName)
+          .map(Path::toString)
+          .sorted()
+          .collect(Collectors.toList());
+    }
+  }
+}