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:57 UTC

[solr] branch branch_9x updated (db6218d0fe1 -> 91a2e84a644)

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

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


    from db6218d0fe1 SOLR-16588: set default knn algorithm (#1255)
     new e4b066ee516 DocRouter: strengthen abstraction (#1215)
     new 9ca10b76755 SOLR-15787: FileSystemConfigSetService: implement fully (#146)
     new 91a2e84a644 SOLR-15787: Fix FileSystemConfigSetService test failure on Windows (#1239)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 solr/CHANGES.txt                                   |  17 +-
 .../org/apache/solr/cloud/ZkConfigSetService.java  |   2 +-
 .../solr/cloud/api/collections/MigrateCmd.java     |   5 +-
 .../org/apache/solr/core/ConfigSetService.java     |  12 +-
 .../solr/core/FileSystemConfigSetService.java      | 190 +++++++++++++++++++--
 .../org/apache/solr/core/backup/BackupManager.java |   8 +-
 .../org/apache/solr/handler/admin/SplitOp.java     |  63 +++----
 .../org/apache/solr/update/SolrIndexSplitter.java  |  22 +--
 .../processor/DistributedZkUpdateProcessor.java    |   3 +-
 .../org/apache/solr/core/TestConfigSetService.java | 125 ++++++++++++++
 .../solr/core/TestFileSystemConfigSetService.java  | 115 +++++++++++++
 .../solr/handler/admin/SplitHandlerTest.java       |   2 +-
 .../solr/common/cloud/CompositeIdRouter.java       |  68 +++++++-
 .../org/apache/solr/common/cloud/DocRouter.java    |   8 +-
 14 files changed, 548 insertions(+), 92 deletions(-)
 create mode 100644 solr/core/src/test/org/apache/solr/core/TestConfigSetService.java
 create mode 100644 solr/core/src/test/org/apache/solr/core/TestFileSystemConfigSetService.java


[solr] 03/03: SOLR-15787: Fix FileSystemConfigSetService test failure on Windows (#1239)

Posted by ds...@apache.org.
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 91a2e84a644ea149361fc56519720064d2b1562c
Author: Nazerke Seidan <se...@gmail.com>
AuthorDate: Mon Jan 9 14:40:49 2023 -0500

    SOLR-15787: Fix FileSystemConfigSetService test failure on Windows (#1239)
    
    Co-authored-by: Nazerke Seidan <ns...@salesforce.com>
    Co-authored-by: Kevin Risden <kr...@apache.org>
---
 .../org/apache/solr/core/ConfigSetService.java     |  8 ++++----
 .../solr/core/FileSystemConfigSetService.java      | 24 +++++++++++++++-------
 .../org/apache/solr/core/backup/BackupManager.java |  8 +++++++-
 .../org/apache/solr/core/TestConfigSetService.java | 19 +++++++----------
 4 files changed, 35 insertions(+), 24 deletions(-)

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 5d9fbf30054..da8c8b90877 100644
--- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java
@@ -399,7 +399,7 @@ public abstract class ConfigSetService {
    * to true then file will be overwritten
    *
    * @param configName the name to give the config
-   * @param fileName the name of the file
+   * @param fileName the name of the file with '/' used as the file path separator
    * @param data the content of the file
    * @param overwriteOnExists if true then file will be overwritten
    * @throws SolrException if file exists and overwriteOnExists == false
@@ -420,7 +420,7 @@ public abstract class ConfigSetService {
    * Download a file from config If the file does not exist, it returns null
    *
    * @param configName the name of the config
-   * @param filePath the file to download
+   * @param filePath the file to download with '/' as the separator
    * @return the content of the file
    */
   public abstract byte[] downloadFileFromConfig(String configName, String filePath)
@@ -453,7 +453,7 @@ public abstract class ConfigSetService {
    * Delete files in config
    *
    * @param configName the name of the config
-   * @param filesToDelete a list of file paths to delete
+   * @param filesToDelete a list of file paths to delete using '/' as file path separator
    */
   public abstract void deleteFilesFromConfig(String configName, List<String> filesToDelete)
       throws IOException;
@@ -488,7 +488,7 @@ public abstract class ConfigSetService {
    * lexicographically e.g. solrconfig.xml, lang/, lang/stopwords_en.txt
    *
    * @param configName the config name
-   * @return list of file name paths in the config
+   * @return list of file name paths in the config with '/' uses as file path separators
    */
   public abstract List<String> getAllConfigFiles(String configName) throws IOException;
 
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 01dcdf8f524..75d8e288834 100644
--- a/solr/core/src/java/org/apache/solr/core/FileSystemConfigSetService.java
+++ b/solr/core/src/java/org/apache/solr/core/FileSystemConfigSetService.java
@@ -94,7 +94,7 @@ public class FileSystemConfigSetService extends ConfigSetService {
     Path configDir = getConfigDir(configName);
     Objects.requireNonNull(filesToDelete);
     for (String fileName : filesToDelete) {
-      Path file = configDir.resolve(fileName);
+      Path file = configDir.resolve(normalizePathToOsSeparator(fileName));
       if (Files.exists(file)) {
         if (Files.isDirectory(file)) {
           deleteDir(file);
@@ -146,7 +146,7 @@ public class FileSystemConfigSetService extends ConfigSetService {
   public void uploadFileToConfig(
       String configName, String fileName, byte[] data, boolean overwriteOnExists)
       throws IOException {
-    Path filePath = getConfigDir(configName).resolve(fileName);
+    Path filePath = getConfigDir(configName).resolve(normalizePathToOsSeparator(fileName));
     if (!Files.exists(filePath) || overwriteOnExists) {
       Files.write(filePath, data);
     }
@@ -218,7 +218,7 @@ public class FileSystemConfigSetService extends ConfigSetService {
 
   @Override
   public byte[] downloadFileFromConfig(String configName, String fileName) throws IOException {
-    Path filePath = getConfigDir(configName).resolve(fileName);
+    Path filePath = getConfigDir(configName).resolve(normalizePathToOsSeparator(fileName));
     byte[] data = null;
     try {
       data = Files.readAllBytes(filePath);
@@ -234,13 +234,13 @@ public class FileSystemConfigSetService extends ConfigSetService {
     List<String> filePaths = new ArrayList<>();
     Files.walkFileTree(
         configDir,
-        new SimpleFileVisitor<Path>() {
+        new SimpleFileVisitor<>() {
           @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());
+            if (!Files.isHidden(file) && !METADATA_FILE.equals(file.getFileName().toString())) {
+              filePaths.add(normalizePathToForwardSlash(configDir.relativize(file).toString()));
               return FileVisitResult.CONTINUE;
             }
             return FileVisitResult.CONTINUE;
@@ -250,7 +250,9 @@ public class FileSystemConfigSetService extends ConfigSetService {
           public FileVisitResult postVisitDirectory(Path dir, IOException ioException) {
             String relativePath = configDir.relativize(dir).toString();
             if (!relativePath.isEmpty()) {
-              filePaths.add(relativePath + "/");
+              // We always want to have a trailing forward slash on a directory to
+              // match the normalization to forward slashes everywhere.
+              filePaths.add(relativePath + '/');
             }
             return FileVisitResult.CONTINUE;
           }
@@ -259,6 +261,14 @@ public class FileSystemConfigSetService extends ConfigSetService {
     return filePaths;
   }
 
+  private String normalizePathToForwardSlash(String path) {
+    return path.replace(configSetBase.getFileSystem().getSeparator(), "/");
+  }
+
+  private String normalizePathToOsSeparator(String path) {
+    return path.replace("/", configSetBase.getFileSystem().getSeparator());
+  }
+
   protected Path locateInstanceDir(CoreDescriptor cd) {
     String configSet = cd.getConfigSet();
     if (configSet == null) return cd.getInstanceDir();
diff --git a/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java b/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
index faae2b9a70c..0d6cd77f3df 100644
--- a/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
@@ -17,6 +17,7 @@
 package org.apache.solr.core.backup;
 
 import com.google.common.base.Preconditions;
+import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
@@ -329,10 +330,15 @@ public class BackupManager {
   private void downloadConfigToRepo(ConfigSetService configSetService, String configName, URI dir)
       throws IOException {
     List<String> filePaths = configSetService.getAllConfigFiles(configName);
+    // getAllConfigFiles always separates file paths with '/'
     for (String filePath : filePaths) {
-      URI uri = repository.resolve(dir, filePath);
+      // Replace '/' to ensure that propre file is resolved for writing.
+      URI uri = repository.resolve(dir, filePath.replace('/', File.separatorChar));
+      // checking for '/' is correct for a directory since ConfigSetService#getAllConfigFiles
+      // always separates file paths with '/'
       if (!filePath.endsWith("/")) {
         log.debug("Writing file {}", filePath);
+        // ConfigSetService#downloadFileFromConfig requires '/' in fle path separator
         byte[] data = configSetService.downloadFileFromConfig(configName, filePath);
         try (OutputStream os = repository.createOutput(uri)) {
           os.write(data);
diff --git a/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java b/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java
index 07cc09b91a2..e1f3787fc88 100644
--- a/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java
+++ b/solr/core/src/test/org/apache/solr/core/TestConfigSetService.java
@@ -30,7 +30,6 @@ 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;
@@ -39,18 +38,15 @@ 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();
     }
@@ -62,11 +58,10 @@ public class TestConfigSetService extends SolrTestCaseJ4 {
   }
 
   @ParametersFactory
-  @SuppressWarnings("rawtypes")
-  public static Iterable<Supplier[]> parameters() {
+  public static Iterable<Supplier<?>[]> parameters() {
     return Arrays.asList(
-        new Supplier[][] {
-          {() -> new ZkConfigSetService(zkClient)},
+        new Supplier<?>[][] {
+          {() -> new ZkConfigSetService(zkServer.getZkClient())},
           {() -> new FileSystemConfigSetService(createTempDir("configsets"))}
         });
   }
@@ -109,15 +104,15 @@ public class TestConfigSetService extends SolrTestCaseJ4 {
 
     List<String> configFiles = configSetService.getAllConfigFiles(configName);
     assertEquals(
-        configFiles.toString(),
-        "[file1, file2, solrconfig.xml, subdir/, subdir/file3, subdir/file4]");
+        configFiles,
+        List.of("file1", "file2", "solrconfig.xml", "subdir/", "subdir/file3", "subdir/file4"));
 
     List<String> configs = configSetService.listConfigs();
-    assertEquals(configs.toString(), "[testconfig]");
+    assertEquals(configs, List.of("testconfig"));
 
     configSetService.copyConfig(configName, "testconfig.AUTOCREATED");
     List<String> copiedConfigFiles = configSetService.getAllConfigFiles("testconfig.AUTOCREATED");
-    assertEquals(configFiles.toString(), (copiedConfigFiles.toString()));
+    assertEquals(configFiles, copiedConfigFiles);
 
     assertEquals(2, configSetService.listConfigs().size());
 


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

Posted by ds...@apache.org.
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());
+    }
+  }
+}


[solr] 01/03: DocRouter: strengthen abstraction (#1215)

Posted by ds...@apache.org.
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 e4b066ee5168df0e72ba5ad4e97ac206e7ee9a17
Author: David Smiley <ds...@salesforce.com>
AuthorDate: Mon Jan 9 17:38:22 2023 -0500

    DocRouter: strengthen abstraction (#1215)
    
    Background: DocRouter is an abstraction with 3 implementations; it isn't pluggable.  There are a number of spots, especially with splits, that were making assumptions of how CompositeIdRouter parsed IDs.
    
    Strengthen the separation of concerns so that the split code can do its job, delegating to the DocRouter (specifically a CompositeIdRouter) on how to parse doc IDs.  It also makes CompositeIdRouter more extendable, but don't add subclasses or plug-ability.
---
 .../solr/cloud/api/collections/MigrateCmd.java     |  5 +-
 .../org/apache/solr/handler/admin/SplitOp.java     | 63 ++++++++++----------
 .../org/apache/solr/update/SolrIndexSplitter.java  | 22 +++----
 .../processor/DistributedZkUpdateProcessor.java    |  3 +-
 .../solr/handler/admin/SplitHandlerTest.java       |  2 +-
 .../solr/common/cloud/CompositeIdRouter.java       | 68 ++++++++++++++++++++--
 .../org/apache/solr/common/cloud/DocRouter.java    |  8 +--
 7 files changed, 112 insertions(+), 59 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
index 70c45027ae4..4ee59d13ca9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
@@ -53,7 +53,6 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
-import org.apache.solr.update.SolrIndexSplitter;
 import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -253,7 +252,7 @@ public class MigrateCmd implements CollApiCmds.CollectionApiCommand {
             SHARD_ID_PROP,
             sourceSlice.getName(),
             "routeKey",
-            SolrIndexSplitter.getRouteKey(splitKey) + "!",
+            sourceRouter.getRouteKeyNoSuffix(splitKey) + "!",
             "range",
             splitRange.toString(),
             "targetCollection",
@@ -283,7 +282,7 @@ public class MigrateCmd implements CollApiCmds.CollectionApiCommand {
       sourceSlice = sourceCollection.getSlice(sourceSlice.getName());
       Map<String, RoutingRule> rules = sourceSlice.getRoutingRules();
       if (rules != null) {
-        RoutingRule rule = rules.get(SolrIndexSplitter.getRouteKey(splitKey) + "!");
+        RoutingRule rule = rules.get(sourceRouter.getRouteKeyNoSuffix(splitKey) + "!");
         if (rule != null && rule.getRouteRanges().contains(splitRange)) {
           added = true;
           break;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java b/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java
index 646b19e2dc2..3b66794c5d5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java
@@ -263,8 +263,9 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
         DocCollection collection = clusterState.getCollection(collectionName);
         String sliceName = parentCore.getCoreDescriptor().getCloudDescriptor().getShardId();
         Slice slice = collection.getSlice(sliceName);
-        DocRouter router =
-            collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
+        CompositeIdRouter router =
+            (CompositeIdRouter)
+                (collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT);
         DocRouter.Range currentRange = slice.getRange();
 
         Object routerObj =
@@ -354,7 +355,10 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
    * Returns a list of range counts sorted by the range lower bound
    */
   static Collection<RangeCount> getHashHistogram(
-      SolrIndexSearcher searcher, String prefixField, DocRouter router, DocCollection collection)
+      SolrIndexSearcher searcher,
+      String prefixField,
+      CompositeIdRouter router,
+      DocCollection collection)
       throws IOException {
     RTimer timer = new RTimer();
     TreeMap<DocRouter.Range, RangeCount> counts = new TreeMap<>();
@@ -374,14 +378,18 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
     while ((term = termsEnum.next()) != null) {
       numPrefixes++;
 
-      String termStr = term.utf8ToString();
-      int firstSep = termStr.indexOf(CompositeIdRouter.SEPARATOR);
       // truncate to first separator since we don't support multiple levels currently
       // NOTE: this does not currently work for tri-level composite ids since the number of bits
       // allocated to the first ID is 16 for a 2 part id and 8 for a 3 part id!
-      if (firstSep != termStr.length() - 1 && firstSep > 0) {
-        numTriLevel++;
-        termStr = termStr.substring(0, firstSep + 1);
+      String termStr;
+      int routeKeyLen = router.getRouteKeyWithSeparator(term.bytes, term.offset, term.length);
+      if (routeKeyLen == 0 || routeKeyLen == term.length) {
+        termStr = term.utf8ToString();
+      } else {
+        int prevLen = term.length;
+        term.length = routeKeyLen;
+        termStr = term.utf8ToString();
+        term.length = prevLen; // restore    (Question: must we do this?)
       }
 
       DocRouter.Range range = router.getSearchRangeSingle(termStr, null, collection);
@@ -418,7 +426,10 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
    * (i.e. the terms are full IDs, not just prefixes)
    */
   static Collection<RangeCount> getHashHistogramFromId(
-      SolrIndexSearcher searcher, String idField, DocRouter router, DocCollection collection)
+      SolrIndexSearcher searcher,
+      String idField,
+      CompositeIdRouter router,
+      DocCollection collection)
       throws IOException {
     RTimer timer = new RTimer();
 
@@ -433,9 +444,8 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
     int numCollisions = 0;
     long sumBuckets = 0;
 
-    byte sep = (byte) CompositeIdRouter.SEPARATOR.charAt(0);
     TermsEnum termsEnum = terms.iterator();
-    BytesRef currPrefix = new BytesRef(); // prefix of the previous "id" term
+    BytesRef currPrefix = new BytesRef(); // prefix of the previous "id" term WITH SEPARATOR
     int bucketCount = 0; // count of the number of docs in the current bucket
 
     // We're going to iterate over all terms, so do the minimum amount of work per term.
@@ -445,7 +455,9 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
       BytesRef term = termsEnum.next();
 
       // compare to current prefix bucket and see if this new term shares the same prefix
-      if (term != null && term.length >= currPrefix.length && currPrefix.length > 0) {
+      if (term != null && currPrefix.length > 0) {
+        // since currPrefix includes the trailing separator, we can assume startsWith is a
+        // sufficient test
         if (StringHelper.startsWith(term, currPrefix)) {
           bucketCount++; // use 1 since we are dealing with unique ids
           continue;
@@ -474,25 +486,16 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
       // if the current term is null, we ran out of values
       if (term == null) break;
 
-      // find the new prefix (if any)
-
-      // resize if needed
-      if (currPrefix.length < term.length) {
-        currPrefix.bytes = new byte[term.length + 10];
-      }
-
-      // Copy the bytes up to and including the separator, and set the length if the separator is
-      // found. If there was no separator, then length remains 0 and it's the indicator that we have
-      // no prefix bucket
-      currPrefix.length = 0;
-      for (int i = 0; i < term.length; i++) {
-        byte b = term.bytes[i + term.offset];
-        currPrefix.bytes[i] = b;
-        if (b == sep) {
-          currPrefix.length = i + 1;
-          bucketCount++;
-          break;
+      // find the new prefix (if any), with trailing separator
+      currPrefix.length = router.getRouteKeyWithSeparator(term.bytes, term.offset, term.length);
+      if (currPrefix.length > 0) {
+        // resize if needed
+        if (currPrefix.length > currPrefix.bytes.length) {
+          currPrefix.bytes = new byte[currPrefix.length + 10];
         }
+        System.arraycopy(term.bytes, term.offset, currPrefix.bytes, 0, currPrefix.length);
+
+        bucketCount++;
       }
     }
 
diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java b/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
index a285f79e73f..61ba505ddd5 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
@@ -133,7 +133,8 @@ public class SolrIndexSplitter {
     if (cmd.splitKey == null) {
       splitKey = null;
     } else {
-      splitKey = getRouteKey(cmd.splitKey);
+      checkRouterSupportsSplitKey(hashRouter, cmd.splitKey);
+      splitKey = ((CompositeIdRouter) hashRouter).getRouteKeyNoSuffix(cmd.splitKey);
     }
     if (cmd.cores == null) {
       this.splitMethod = SplitMethod.REWRITE;
@@ -647,6 +648,7 @@ public class SolrIndexSplitter {
       AtomicInteger currentPartition,
       boolean delete)
       throws IOException {
+    checkRouterSupportsSplitKey(hashRouter, splitKey);
     LeafReader reader = readerContext.reader();
     FixedBitSet[] docSets = new FixedBitSet[numPieces];
     for (int i = 0; i < docSets.length; i++) {
@@ -689,8 +691,7 @@ public class SolrIndexSplitter {
       String idString = idRef.toString();
 
       if (splitKey != null) {
-        // todo have composite routers support these kind of things instead
-        String part1 = getRouteKey(idString);
+        String part1 = ((CompositeIdRouter) hashRouter).getRouteKeyNoSuffix(idString);
         if (part1 == null) continue;
         if (!splitKey.equals(part1)) {
           continue;
@@ -765,18 +766,11 @@ public class SolrIndexSplitter {
     return docSets;
   }
 
-  public static String getRouteKey(String idString) {
-    int idx = idString.indexOf(CompositeIdRouter.SEPARATOR);
-    if (idx <= 0) return null;
-    String part1 = idString.substring(0, idx);
-    int commaIdx = part1.indexOf(CompositeIdRouter.bitsSeparator);
-    if (commaIdx > 0 && commaIdx + 1 < part1.length()) {
-      char ch = part1.charAt(commaIdx + 1);
-      if (ch >= '0' && ch <= '9') {
-        part1 = part1.substring(0, commaIdx);
-      }
+  private static void checkRouterSupportsSplitKey(HashBasedRouter hashRouter, String splitKey) {
+    if (splitKey != null && !(hashRouter instanceof CompositeIdRouter)) {
+      throw new IllegalStateException(
+          "splitKey isn't supported for router " + hashRouter.getClass());
     }
-    return part1;
   }
 
   // change livedocs on the reader to delete those docs we don't want
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
index fbe89388192..c1eee8572c2 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
@@ -67,7 +67,6 @@ import org.apache.solr.update.DeleteUpdateCommand;
 import org.apache.solr.update.MergeIndexesCommand;
 import org.apache.solr.update.RollbackUpdateCommand;
 import org.apache.solr.update.SolrCmdDistributor;
-import org.apache.solr.update.SolrIndexSplitter;
 import org.apache.solr.update.UpdateCommand;
 import org.apache.solr.util.TestInjection;
 import org.apache.zookeeper.KeeperException;
@@ -1067,7 +1066,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
           return nodes;
         }
 
-        String routeKey = SolrIndexSplitter.getRouteKey(id);
+        String routeKey = compositeIdRouter.getRouteKeyNoSuffix(id);
         if (routeKey != null) {
           RoutingRule rule = routingRules.get(routeKey + "!");
           if (rule != null) {
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/SplitHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/SplitHandlerTest.java
index f3f1f69322f..011f5fe3885 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/SplitHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/SplitHandlerTest.java
@@ -239,7 +239,7 @@ public class SplitHandlerTest extends SolrTestCaseJ4 {
 
     String prefixField = "id_prefix_s";
     String idField = "id";
-    DocRouter router = new CompositeIdRouter();
+    CompositeIdRouter router = new CompositeIdRouter();
 
     for (int i = 0; i < 100; i++) {
       SolrQueryRequest req = req("myquery");
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java b/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
index dd0b6669195..9094ddd84e8 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/CompositeIdRouter.java
@@ -81,12 +81,57 @@ import org.apache.solr.common.util.Hash;
 public class CompositeIdRouter extends HashBasedRouter {
   public static final String NAME = "compositeId";
 
-  public static final String SEPARATOR = "!";
+  // TODO standardize naming: routeKey (probably) or shardKey or key; pick one.
+
+  /**
+   * This character separates a composite ID into a leading route key and the rest.
+   *
+   * <p>Importantly, it's also used at the end of a provided route key parameter (which appears in
+   * many places) to designate a hash range which translates to a list of slices. If a route key
+   * does not end with this character, then semantically the key points to a single slice that holds
+   * a doc with that ID.
+   */
+  public static final char SEPARATOR = '!';
 
   // separator used to optionally specify number of bits to allocate toward first part.
   public static final int bitsSeparator = '/';
   private int bits = 16;
 
+  /**
+   * Parse out the route key from {@code id} up to and including the {@link #SEPARATOR}, returning
+   * it's length. If no route key is detected then 0 is returned.
+   */
+  public int getRouteKeyWithSeparator(byte[] id, int idOffset, int idLength) {
+    final byte SEPARATOR_BYTE = (byte) CompositeIdRouter.SEPARATOR;
+    for (int i = 0; i < idLength; i++) {
+      byte b = id[idOffset + i];
+      if (b == SEPARATOR_BYTE) {
+        return i + 1;
+      }
+    }
+    return 0;
+  }
+
+  /**
+   * Parse out the route key from {@code id}. It will not have the "bits" suffix or separator, if
+   * present. If there is no route key found then null is returned.
+   */
+  public String getRouteKeyNoSuffix(String id) {
+    int idx = id.indexOf(SEPARATOR);
+    if (idx <= 0) {
+      return null;
+    }
+    String part1 = id.substring(0, idx);
+    int commaIdx = part1.indexOf(bitsSeparator);
+    if (commaIdx > 0 && commaIdx + 1 < part1.length()) {
+      char ch = part1.charAt(commaIdx + 1);
+      if (ch >= '0' && ch <= '9') {
+        part1 = part1.substring(0, commaIdx);
+      }
+    }
+    return part1;
+  }
+
   @Override
   public int sliceHash(
       String id, SolrInputDocument doc, SolrParams params, DocCollection collection) {
@@ -132,6 +177,8 @@ public class CompositeIdRouter extends HashBasedRouter {
    * @return Range for given routeKey
    */
   public Range keyHashRange(String routeKey) {
+    routeKey = preprocessRouteKey(routeKey);
+
     if (routeKey.indexOf(SEPARATOR) < 0) {
       int hash = sliceHash(routeKey, null, null, null);
       return new Range(hash, hash);
@@ -147,6 +194,8 @@ public class CompositeIdRouter extends HashBasedRouter {
       return fullRange();
     }
 
+    shardKey = preprocessRouteKey(shardKey);
+
     if (shardKey.indexOf(SEPARATOR) < 0) {
       // shardKey is a simple id, so don't do a range
       int hash = Hash.murmurhash3_x86_32(shardKey, 0, shardKey.length(), 0);
@@ -164,7 +213,8 @@ public class CompositeIdRouter extends HashBasedRouter {
       // TODO: this may need modification in the future when shard splitting could cause an overlap
       return collection.getActiveSlices();
     }
-    String id = shardKey;
+
+    String id = preprocessRouteKey(shardKey);
 
     if (shardKey.indexOf(SEPARATOR) < 0) {
       // shardKey is a simple id, so don't do a range
@@ -185,6 +235,14 @@ public class CompositeIdRouter extends HashBasedRouter {
     return targetSlices;
   }
 
+  /**
+   * Methods accepting a route key (shard key) can have this input preprocessed by a subclass before
+   * further analysis.
+   */
+  protected String preprocessRouteKey(String shardKey) {
+    return shardKey;
+  }
+
   @Override
   public String getName() {
     return NAME;
@@ -266,7 +324,7 @@ public class CompositeIdRouter extends HashBasedRouter {
   }
 
   /** Helper class to calculate parts, masks etc for an id. */
-  static class KeyParser {
+  protected static class KeyParser {
     String key;
     int[] numBits;
     int[] hashes;
@@ -333,7 +391,7 @@ public class CompositeIdRouter extends HashBasedRouter {
       masks = getMasks();
     }
 
-    Range getRange() {
+    public Range getRange() {
       int lowerBound;
       int upperBound;
 
@@ -395,7 +453,7 @@ public class CompositeIdRouter extends HashBasedRouter {
       return masks;
     }
 
-    int getHash() {
+    public int getHash() {
       int result = hashes[0] & masks[0];
 
       for (int i = 1; i < pieces; i++) result = result | (hashes[i] & masks[i]);
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java
index af0ebc66f12..490a0a21eda 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java
@@ -39,7 +39,7 @@ import org.noggit.JSONWriter;
  */
 public abstract class DocRouter {
   public static final String DEFAULT_NAME = CompositeIdRouter.NAME;
-  public static final DocRouter DEFAULT = new CompositeIdRouter();
+  public static final DocRouter DEFAULT;
 
   public static DocRouter getDocRouter(String routerName) {
     DocRouter router = routerMap.get(routerName);
@@ -79,11 +79,11 @@ public abstract class DocRouter {
     // to "plain" if it doesn't have any properties.
     routerMap.put(null, plain); // back compat with 4.0
     routerMap.put(PlainIdRouter.NAME, plain);
-    routerMap.put(
-        CompositeIdRouter.NAME,
-        DEFAULT_NAME.equals(CompositeIdRouter.NAME) ? DEFAULT : new CompositeIdRouter());
+    routerMap.put(CompositeIdRouter.NAME, new CompositeIdRouter());
     routerMap.put(ImplicitDocRouter.NAME, new ImplicitDocRouter());
     // NOTE: careful that the map keys (the static .NAME members) are filled in by making them final
+
+    DEFAULT = routerMap.get(DEFAULT_NAME);
   }
 
   // Hash ranges can't currently "wrap" - i.e. max must be greater or equal to min.