You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by br...@apache.org on 2024/02/06 15:07:14 UTC

(solr) branch main updated: SOLR-17146: Add DelegatingBackupRepository and alternative checksum verification (#2239)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new fa8149564cc SOLR-17146: Add DelegatingBackupRepository and alternative checksum verification (#2239)
fa8149564cc is described below

commit fa8149564cc8681c7ee77f25e667401c82be1c44
Author: Bruno Roustant <33...@users.noreply.github.com>
AuthorDate: Tue Feb 6 16:07:08 2024 +0100

    SOLR-17146: Add DelegatingBackupRepository and alternative checksum verification (#2239)
---
 solr/CHANGES.txt                                   |   2 +
 .../repository/AbstractBackupRepository.java       |  77 ++++++++
 .../core/backup/repository/BackupRepository.java   |  33 ++++
 .../backup/repository/BackupRepositoryFactory.java |  41 +++--
 .../repository/DelegatingBackupRepository.java}    |  94 +++++-----
 .../repository/LocalFileSystemRepository.java      |  12 +-
 .../repository/LocalFileSystemRepositoryTest.java  |  50 ++++++
 .../org/apache/solr/gcs/GCSBackupRepository.java   |  44 +++--
 .../apache/solr/gcs/GCSBackupRepositoryTest.java   |  15 +-
 .../backup/repository/HdfsBackupRepository.java    |  11 +-
 .../HdfsBackupRepositoryIntegrationTest.java       |   5 +
 .../org/apache/solr/s3/S3BackupRepository.java     |  24 ++-
 .../apache/solr/s3/S3BackupRepositoryConfig.java   |  10 +-
 .../org/apache/solr/s3/S3BackupRepositoryTest.java |   5 +
 .../deployment-guide/pages/backup-restore.adoc     |  15 +-
 .../collections/AbstractBackupRepositoryTest.java  | 194 ++++++++++++++++++++-
 .../apache/solr/core/TrackingBackupRepository.java | 101 +----------
 17 files changed, 519 insertions(+), 214 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index db542ef7db8..a6b4567eaa7 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -93,6 +93,8 @@ Improvements
 * SOLR-17038: Expose the term count in /admin/segments handler if fieldInfo=true.
   (Rahul Goswami via David Smiley and Christine Poerschke)
 
+* SOLR-17146: Add DelegatingBackupRepository and alternative checksum verification (Bruno Roustant)
+
 Optimizations
 ---------------------
 * SOLR-17144: Close searcherExecutor thread per core after 1 minute (Pierre Salagnac, Christine Poerschke)
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/AbstractBackupRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/AbstractBackupRepository.java
new file mode 100644
index 00000000000..f58983491bb
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/AbstractBackupRepository.java
@@ -0,0 +1,77 @@
+/*
+ * 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.backup.repository;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.common.util.NamedList;
+
+/**
+ * Abstract {@link BackupRepository} implementation providing some logic shared between real
+ * implementations.
+ */
+public abstract class AbstractBackupRepository implements BackupRepository {
+
+  /**
+   * Plugin initialization parameter to define whether the {@link BackupRepository} should verify
+   * the checksum before copying index files. Defaults to {@code true}.
+   *
+   * <p>If the checksum cannot be verified in the standard Lucene way ({@link
+   * CodecUtil#checkFooter}, then this parameter can be set to false, and the checksum should be
+   * verified in a specific way.
+   */
+  public static final String PARAM_VERIFY_CHECKSUM = "verifyChecksum";
+
+  protected NamedList<?> config;
+  protected boolean shouldVerifyChecksum;
+
+  @Override
+  public void init(NamedList<?> args) {
+    config = args;
+    shouldVerifyChecksum = getBooleanConfig(args, PARAM_VERIFY_CHECKSUM, true);
+  }
+
+  /**
+   * Copies an index file from a specified {@link Directory} to a destination {@link Directory}.
+   * Also verifies the checksum unless {@link #PARAM_VERIFY_CHECKSUM} was false in the {@link
+   * #init(NamedList)} arguments.
+   *
+   * @param sourceDir The source directory hosting the file to be copied.
+   * @param sourceFileName The name of the file to be copied
+   * @param destDir The destination directory.
+   * @throws CorruptIndexException in case checksum of the file does not match with precomputed
+   *     checksum stored at the end of the file
+   */
+  @Override
+  public void copyIndexFileFrom(
+      Directory sourceDir, String sourceFileName, Directory destDir, String destFileName)
+      throws IOException {
+    if (shouldVerifyChecksum) {
+      BackupRepository.super.copyIndexFileFrom(sourceDir, sourceFileName, destDir, destFileName);
+    } else {
+      copyFileNoChecksum(sourceDir, sourceFileName, destDir, destFileName);
+    }
+  }
+
+  protected static boolean getBooleanConfig(NamedList<?> args, String param, boolean defaultValue) {
+    Object value = args.get(param);
+    return value == null ? defaultValue : Boolean.parseBoolean(value.toString());
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java
index 3879b59d281..68cf5636fdd 100644
--- a/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java
@@ -204,6 +204,15 @@ public interface BackupRepository extends NamedListInitializedPlugin, Closeable
     }
   }
 
+  /**
+   * Copies an index file from a specified {@link Directory} to a destination {@link Directory}.
+   *
+   * @param sourceDir The source directory hosting the file to be copied.
+   * @param sourceFileName The name of the file to be copied
+   * @param destDir The destination directory.
+   * @throws CorruptIndexException in case checksum of the file does not match with precomputed
+   *     checksum stored at the end of the file
+   */
   default void copyIndexFileFrom(
       Directory sourceDir, String sourceFileName, Directory destDir, String destFileName)
       throws IOException {
@@ -277,4 +286,28 @@ public interface BackupRepository extends NamedListInitializedPlugin, Closeable
       throws IOException {
     throw new UnsupportedOperationException();
   }
+
+  /**
+   * Copy a file from a source {@link Directory} to a destination {@link Directory} without
+   * verifying the checksum.
+   *
+   * @param sourceDir The source directory hosting the file to be copied.
+   * @param sourceFileName The name of the file to be copied.
+   * @param destDir The destination directory to copy the file to.
+   * @param destFileName The name of the copied file at destination.
+   */
+  default void copyFileNoChecksum(
+      Directory sourceDir, String sourceFileName, Directory destDir, String destFileName)
+      throws IOException {
+    boolean success = false;
+    try (IndexInput is = sourceDir.openInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+        IndexOutput os = destDir.createOutput(destFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)) {
+      os.copyBytes(is, is.length());
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.deleteFilesIgnoringExceptions(destDir, destFileName);
+      }
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java
index 5d7be935d06..c838ebd4185 100644
--- a/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java
@@ -17,6 +17,8 @@
 
 package org.apache.solr.core.backup.repository;
 
+import static org.apache.solr.core.backup.repository.DelegatingBackupRepository.PARAM_DELEGATE_REPOSITORY_NAME;
+
 import java.lang.invoke.MethodHandles;
 import java.util.HashMap;
 import java.util.Map;
@@ -70,25 +72,34 @@ public class BackupRepositoryFactory {
   public BackupRepository newInstance(SolrResourceLoader loader, String name) {
     Objects.requireNonNull(loader);
     Objects.requireNonNull(name);
-    PluginInfo repo =
-        Objects.requireNonNull(
-            backupRepoPluginByName.get(name),
-            "Could not find a backup repository with name " + name);
+    PluginInfo repo = getBackupRepoPlugin(name);
+    BackupRepository backupRepository = loader.newInstance(repo.className, BackupRepository.class);
+    backupRepository.init(repo.initArgs);
 
-    BackupRepository result = loader.newInstance(repo.className, BackupRepository.class);
-    if ("trackingBackupRepository".equals(name)) {
-      // newInstance can be called by multiple threads, synchronization prevents simultaneous
-      // multithreaded 'adds' from corrupting the NamedList
-      synchronized (repo.initArgs) {
-        if (repo.initArgs.get("factory") == null) {
-          repo.initArgs.add("factory", this);
-          repo.initArgs.add("loader", loader);
-        }
+    if (backupRepository instanceof DelegatingBackupRepository) {
+      DelegatingBackupRepository delegatingRepo = (DelegatingBackupRepository) backupRepository;
+      String delegateName = (String) repo.initArgs.get(PARAM_DELEGATE_REPOSITORY_NAME);
+      if (delegateName == null) {
+        throw new SolrException(
+            ErrorCode.SERVER_ERROR,
+            "Missing '"
+                + PARAM_DELEGATE_REPOSITORY_NAME
+                + "' parameter for backup repository with name "
+                + name);
       }
+      PluginInfo delegatePlugin = getBackupRepoPlugin(delegateName);
+      BackupRepository delegate =
+          loader.newInstance(delegatePlugin.className, BackupRepository.class);
+      delegate.init(delegatingRepo.getDelegateInitArgs(delegatePlugin.initArgs));
+      delegatingRepo.setDelegate(delegate);
     }
 
-    result.init(repo.initArgs);
-    return result;
+    return backupRepository;
+  }
+
+  private PluginInfo getBackupRepoPlugin(String name) {
+    return Objects.requireNonNull(
+        backupRepoPluginByName.get(name), "Could not find a backup repository with name " + name);
   }
 
   public BackupRepository newInstance(SolrResourceLoader loader) {
diff --git a/solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/DelegatingBackupRepository.java
similarity index 67%
copy from solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java
copy to solr/core/src/java/org/apache/solr/core/backup/repository/DelegatingBackupRepository.java
index 5e28d7aff62..a603c208cb8 100644
--- a/solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/DelegatingBackupRepository.java
@@ -14,31 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.solr.core;
+package org.apache.solr.core.backup.repository;
 
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.URI;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.backup.Checksum;
-import org.apache.solr.core.backup.repository.BackupRepository;
-import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
 
-public class TrackingBackupRepository implements BackupRepository {
-  private static final List<URI> COPIED_FILES = Collections.synchronizedList(new ArrayList<>());
-  private static final List<URI> DIRECTORIES_CREATED =
-      Collections.synchronizedList(new ArrayList<>());
-  private static final List<URI> OUTPUTS_CREATED = Collections.synchronizedList(new ArrayList<>());
+/** Delegates to another {@link BackupRepository}. */
+public class DelegatingBackupRepository implements BackupRepository {
+
+  public static final String PARAM_DELEGATE_REPOSITORY_NAME = "delegateRepoName";
+
+  protected BackupRepository delegate;
 
-  private BackupRepository delegate;
+  protected NamedList<?> getDelegateInitArgs(NamedList<?> initArgs) {
+    return initArgs;
+  }
+
+  protected void setDelegate(BackupRepository delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public String getBackupLocation(String override) {
+    return delegate.getBackupLocation(override);
+  }
 
   @Override
   public <T> T getConfigProperty(String name) {
@@ -87,13 +93,11 @@ public class TrackingBackupRepository implements BackupRepository {
 
   @Override
   public OutputStream createOutput(URI path) throws IOException {
-    OUTPUTS_CREATED.add(path);
     return delegate.createOutput(path);
   }
 
   @Override
   public void createDirectory(URI path) throws IOException {
-    DIRECTORIES_CREATED.add(path);
     delegate.createDirectory(path);
   }
 
@@ -103,57 +107,42 @@ public class TrackingBackupRepository implements BackupRepository {
   }
 
   @Override
-  public void copyIndexFileFrom(
-      Directory sourceDir, String sourceFileName, URI destDir, String destFileName)
-      throws IOException {
-    COPIED_FILES.add(delegate.resolve(destDir, destFileName));
-    delegate.copyIndexFileFrom(sourceDir, sourceFileName, destDir, destFileName);
+  public void copyFileFrom(Directory sourceDir, String fileName, URI dest) throws IOException {
+    delegate.copyFileFrom(sourceDir, fileName, dest);
   }
 
   @Override
-  public void close() throws IOException {
-    delegate.close();
+  public void copyFileTo(URI sourceRepo, String fileName, Directory dest) throws IOException {
+    delegate.copyFileTo(sourceRepo, fileName, dest);
   }
 
   @Override
-  public void delete(URI path, Collection<String> files) throws IOException {
-    delegate.delete(path, files);
+  public String[] listAllOrEmpty(URI path) {
+    return delegate.listAllOrEmpty(path);
   }
 
   @Override
-  public Checksum checksum(Directory dir, String fileName) throws IOException {
-    return delegate.checksum(dir, fileName);
+  public void copyIndexFileFrom(
+      Directory sourceDir, String sourceFileName, Directory destDir, String destFileName)
+      throws IOException {
+    delegate.copyIndexFileFrom(sourceDir, sourceFileName, destDir, destFileName);
   }
 
   @Override
-  public void init(NamedList<?> args) {
-    BackupRepositoryFactory factory = (BackupRepositoryFactory) args.get("factory");
-    SolrResourceLoader loader = (SolrResourceLoader) args.get("loader");
-    String repoName = (String) args.get("delegateRepoName");
-
-    this.delegate = factory.newInstance(loader, repoName);
-  }
-
-  /**
-   * @return list of files were copied by using {@link #copyFileFrom(Directory, String, URI)}
-   */
-  public static List<URI> copiedFiles() {
-    return new ArrayList<>(COPIED_FILES);
-  }
-
-  public static List<URI> directoriesCreated() {
-    return new ArrayList<>(DIRECTORIES_CREATED);
+  public void delete(URI path, Collection<String> files) throws IOException {
+    delegate.delete(path, files);
   }
 
-  public static List<URI> outputsCreated() {
-    return new ArrayList<>(OUTPUTS_CREATED);
+  @Override
+  public Checksum checksum(Directory dir, String fileName) throws IOException {
+    return delegate.checksum(dir, fileName);
   }
 
-  /** Clear all tracking data */
-  public static void clear() {
-    COPIED_FILES.clear();
-    DIRECTORIES_CREATED.clear();
-    OUTPUTS_CREATED.clear();
+  @Override
+  public void copyIndexFileFrom(
+      Directory sourceDir, String sourceFileName, URI destDir, String destFileName)
+      throws IOException {
+    delegate.copyIndexFileFrom(sourceDir, sourceFileName, destDir, destFileName);
   }
 
   @Override
@@ -162,4 +151,9 @@ public class TrackingBackupRepository implements BackupRepository {
       throws IOException {
     delegate.copyIndexFileTo(sourceRepo, sourceFileName, dest, destFileName);
   }
+
+  @Override
+  public void close() throws IOException {
+    delegate.close();
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java
index f1dbe7d7362..77f7e1921f3 100644
--- a/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java
@@ -34,7 +34,6 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.store.NoLockFactory;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.DirectoryFactory;
 
 /**
@@ -42,14 +41,7 @@ import org.apache.solr.core.DirectoryFactory;
  * Solr indexes to a local file-system. (Note - This can even be used for a shared file-system if it
  * is exposed via a local file-system interface e.g. NFS).
  */
-public class LocalFileSystemRepository implements BackupRepository {
-
-  private NamedList<?> config = null;
-
-  @Override
-  public void init(NamedList<?> args) {
-    this.config = args;
-  }
+public class LocalFileSystemRepository extends AbstractBackupRepository {
 
   @SuppressWarnings("unchecked")
   @Override
@@ -97,7 +89,7 @@ public class LocalFileSystemRepository implements BackupRepository {
   public void createDirectory(URI path) throws IOException {
     Path p = Path.of(path);
     if (!Files.exists(p, LinkOption.NOFOLLOW_LINKS)) {
-      Files.createDirectory(p);
+      Files.createDirectories(p);
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/core/backup/repository/LocalFileSystemRepositoryTest.java b/solr/core/src/test/org/apache/solr/core/backup/repository/LocalFileSystemRepositoryTest.java
new file mode 100644
index 00000000000..82b120a447c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/backup/repository/LocalFileSystemRepositoryTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.backup.repository;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import org.apache.solr.cloud.api.collections.AbstractBackupRepositoryTest;
+import org.junit.BeforeClass;
+
+/** {@link LocalFileSystemRepository} test. */
+public class LocalFileSystemRepositoryTest extends AbstractBackupRepositoryTest {
+
+  private static URI baseUri;
+
+  @BeforeClass
+  public static void setupBaseDir() {
+    baseUri = createTempDir().toUri();
+  }
+
+  @Override
+  protected Class<? extends BackupRepository> getRepositoryClass() {
+    return LocalFileSystemRepository.class;
+  }
+
+  @Override
+  protected BackupRepository getRepository() {
+    LocalFileSystemRepository repo = new LocalFileSystemRepository();
+    repo.init(getBaseBackupRepositoryConfiguration());
+    return repo;
+  }
+
+  @Override
+  protected URI getBaseUri() throws URISyntaxException {
+    return baseUri;
+  }
+}
diff --git a/solr/modules/gcs-repository/src/java/org/apache/solr/gcs/GCSBackupRepository.java b/solr/modules/gcs-repository/src/java/org/apache/solr/gcs/GCSBackupRepository.java
index 5f9b2314b5d..13f9f78bcd5 100644
--- a/solr/modules/gcs-repository/src/java/org/apache/solr/gcs/GCSBackupRepository.java
+++ b/solr/modules/gcs-repository/src/java/org/apache/solr/gcs/GCSBackupRepository.java
@@ -54,19 +54,19 @@ import org.apache.lucene.store.IndexOutput;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.backup.repository.AbstractBackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /** {@link BackupRepository} implementation that stores files in Google Cloud Storage ("GCS"). */
-public class GCSBackupRepository implements BackupRepository {
+public class GCSBackupRepository extends AbstractBackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private static final int LARGE_BLOB_THRESHOLD_BYTE_SIZE = 5 * 1024 * 1024;
   private static final Storage.BlobWriteOption[] NO_WRITE_OPTIONS = new Storage.BlobWriteOption[0];
 
   protected Storage storage;
 
-  private NamedList<?> config = null;
   protected String bucketName = null;
   protected String credentialPath = null;
   protected int writeBufferSizeBytes;
@@ -96,7 +96,7 @@ public class GCSBackupRepository implements BackupRepository {
 
   @Override
   public void init(NamedList<?> args) {
-    this.config = args;
+    super.init(args);
     final GCSConfigParser configReader = new GCSConfigParser();
     final GCSConfigParser.GCSConfig parsedConfig = configReader.parseConfiguration(config);
 
@@ -342,8 +342,10 @@ public class GCSBackupRepository implements BackupRepository {
     blobName = appendTrailingSeparatorIfNecessary(blobName);
     blobName += destFileName;
     final BlobInfo blobInfo = BlobInfo.newBuilder(bucketName, blobName).build();
-    try (ChecksumIndexInput input =
-        sourceDir.openChecksumInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)) {
+    try (IndexInput input =
+        shouldVerifyChecksum
+            ? sourceDir.openChecksumInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)
+            : sourceDir.openInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)) {
       if (input.length() <= CodecUtil.footerLength()) {
         throw new CorruptIndexException("file is too small:" + input.length(), input);
       }
@@ -383,14 +385,18 @@ public class GCSBackupRepository implements BackupRepository {
   @Override
   public void close() throws IOException {}
 
-  private void writeBlobMultipart(BlobInfo blobInfo, ChecksumIndexInput indexInput, int blobSize)
+  private void writeBlobMultipart(BlobInfo blobInfo, IndexInput indexInput, int blobSize)
       throws IOException {
     byte[] bytes = new byte[blobSize];
-    indexInput.readBytes(bytes, 0, blobSize - CodecUtil.footerLength());
-    long checksum = CodecUtil.checkFooter(indexInput);
-    ByteBuffer footerBuffer =
-        ByteBuffer.wrap(bytes, blobSize - CodecUtil.footerLength(), CodecUtil.footerLength());
-    writeFooter(checksum, footerBuffer);
+    if (shouldVerifyChecksum) {
+      indexInput.readBytes(bytes, 0, blobSize - CodecUtil.footerLength());
+      long checksum = CodecUtil.checkFooter((ChecksumIndexInput) indexInput);
+      ByteBuffer footerBuffer =
+          ByteBuffer.wrap(bytes, blobSize - CodecUtil.footerLength(), CodecUtil.footerLength());
+      writeFooter(checksum, footerBuffer);
+    } else {
+      indexInput.readBytes(bytes, 0, blobSize);
+    }
     try {
       storage.create(blobInfo, bytes, Storage.BlobTargetOption.doesNotExist());
     } catch (final StorageException se) {
@@ -401,15 +407,17 @@ public class GCSBackupRepository implements BackupRepository {
     }
   }
 
-  private void writeBlobResumable(BlobInfo blobInfo, ChecksumIndexInput indexInput)
-      throws IOException {
+  private void writeBlobResumable(BlobInfo blobInfo, IndexInput indexInput) throws IOException {
     try {
       final WriteChannel writeChannel = storage.writer(blobInfo, getDefaultBlobWriteOptions());
 
       ByteBuffer buffer = ByteBuffer.allocate(writeBufferSizeBytes);
       writeChannel.setChunkSize(writeBufferSizeBytes);
 
-      long remain = indexInput.length() - CodecUtil.footerLength();
+      long remain =
+          shouldVerifyChecksum
+              ? indexInput.length() - CodecUtil.footerLength()
+              : indexInput.length();
       while (remain > 0) {
         // reading
         int byteReads = (int) Math.min(buffer.capacity(), remain);
@@ -422,9 +430,11 @@ public class GCSBackupRepository implements BackupRepository {
         buffer.clear();
         remain -= byteReads;
       }
-      long checksum = CodecUtil.checkFooter(indexInput);
-      ByteBuffer bytes = getFooter(checksum);
-      writeChannel.write(bytes);
+      if (shouldVerifyChecksum) {
+        long checksum = CodecUtil.checkFooter((ChecksumIndexInput) indexInput);
+        ByteBuffer bytes = getFooter(checksum);
+        writeChannel.write(bytes);
+      }
       writeChannel.close();
     } catch (final StorageException se) {
       if (se.getCode() == HTTP_PRECON_FAILED) {
diff --git a/solr/modules/gcs-repository/src/test/org/apache/solr/gcs/GCSBackupRepositoryTest.java b/solr/modules/gcs-repository/src/test/org/apache/solr/gcs/GCSBackupRepositoryTest.java
index 03aa0aff549..10d2acb2bd6 100644
--- a/solr/modules/gcs-repository/src/test/org/apache/solr/gcs/GCSBackupRepositoryTest.java
+++ b/solr/modules/gcs-repository/src/test/org/apache/solr/gcs/GCSBackupRepositoryTest.java
@@ -35,14 +35,25 @@ import org.junit.Test;
 public class GCSBackupRepositoryTest extends AbstractBackupRepositoryTest {
 
   @AfterClass
-  public static void tearDownClass() throws Exception {
+  public static void tearDownClass() {
     LocalStorageGCSBackupRepository.clearStashedStorage();
   }
 
   @Override
-  protected BackupRepository getRepository() {
+  protected Class<? extends BackupRepository> getRepositoryClass() {
+    return LocalStorageGCSBackupRepository.class;
+  }
+
+  @Override
+  protected NamedList<Object> getBaseBackupRepositoryConfiguration() {
     final NamedList<Object> config = new NamedList<>();
     config.add(BACKUP_LOCATION, "backup1");
+    return config;
+  }
+
+  @Override
+  protected BackupRepository getRepository() {
+    NamedList<Object> config = getBaseBackupRepositoryConfiguration();
     final GCSBackupRepository repository = new LocalStorageGCSBackupRepository();
     repository.init(config);
 
diff --git a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/backup/repository/HdfsBackupRepository.java b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/backup/repository/HdfsBackupRepository.java
index 2f02a9fa3fd..87b05bac59e 100644
--- a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/backup/repository/HdfsBackupRepository.java
+++ b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/backup/repository/HdfsBackupRepository.java
@@ -19,7 +19,6 @@ package org.apache.solr.hdfs.backup.repository;
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.lang.invoke.MethodHandles;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Collection;
@@ -37,15 +36,12 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.DirectoryFactory;
-import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.AbstractBackupRepository;
 import org.apache.solr.hdfs.HdfsDirectoryFactory;
 import org.apache.solr.hdfs.store.HdfsDirectory;
 import org.apache.solr.hdfs.store.HdfsDirectory.HdfsIndexInput;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class HdfsBackupRepository implements BackupRepository {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+public class HdfsBackupRepository extends AbstractBackupRepository {
 
   private static final String HDFS_UMASK_MODE_PARAM = "solr.hdfs.permissions.umask-mode";
   private static final String HDFS_COPY_BUFFER_SIZE_PARAM = "solr.hdfs.buffer.size";
@@ -54,12 +50,11 @@ public class HdfsBackupRepository implements BackupRepository {
   private Configuration hdfsConfig = null;
   private FileSystem fileSystem = null;
   private Path baseHdfsPath = null;
-  private NamedList<?> config = null;
   protected int copyBufferSize = HdfsDirectory.DEFAULT_BUFFER_SIZE;
 
   @Override
   public void init(NamedList<?> args) {
-    this.config = args;
+    super.init(args);
 
     // Configure the size of the buffer used for copying index files to/from HDFS, if specified.
     if (args.get(HDFS_COPY_BUFFER_SIZE_PARAM) != null) {
diff --git a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/backup/repository/HdfsBackupRepositoryIntegrationTest.java b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/backup/repository/HdfsBackupRepositoryIntegrationTest.java
index ecd5349de3f..525bc7c00f7 100644
--- a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/backup/repository/HdfsBackupRepositoryIntegrationTest.java
+++ b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/backup/repository/HdfsBackupRepositoryIntegrationTest.java
@@ -100,6 +100,11 @@ public class HdfsBackupRepositoryIntegrationTest extends AbstractBackupRepositor
     }
   }
 
+  @Override
+  protected Class<? extends BackupRepository> getRepositoryClass() {
+    return HdfsBackupRepository.class;
+  }
+
   @Override
   protected BackupRepository getRepository() {
     HdfsBackupRepository repository = new HdfsBackupRepository();
diff --git a/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepository.java b/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepository.java
index 728c36a3b56..122dcc11918 100644
--- a/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepository.java
+++ b/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepository.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.backup.repository.AbstractBackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,26 +48,24 @@ import org.slf4j.LoggerFactory;
  * A concrete implementation of {@link BackupRepository} interface supporting backup/restore of Solr
  * indexes to S3.
  */
-public class S3BackupRepository implements BackupRepository {
+public class S3BackupRepository extends AbstractBackupRepository {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final int CHUNK_SIZE = 16 * 1024 * 1024; // 16 MBs
   static final String S3_SCHEME = "s3";
 
-  private NamedList<?> config;
   private S3StorageClient client;
 
   @Override
   public void init(NamedList<?> args) {
-    this.config = args;
+    super.init(args);
     S3BackupRepositoryConfig backupConfig = new S3BackupRepositoryConfig(this.config);
 
     // If a client was already created, close it to avoid any resource leak
     if (client != null) {
       client.close();
     }
-
     this.client = backupConfig.buildClient();
   }
 
@@ -279,8 +278,10 @@ public class S3BackupRepository implements BackupRepository {
       log.debug("Upload started to S3 '{}'", s3Path);
     }
 
-    try (ChecksumIndexInput indexInput =
-        sourceDir.openChecksumInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)) {
+    try (IndexInput indexInput =
+        shouldVerifyChecksum
+            ? sourceDir.openChecksumInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)
+            : sourceDir.openInput(sourceFileName, DirectoryFactory.IOCONTEXT_NO_CACHE)) {
       if (indexInput.length() <= CodecUtil.footerLength()) {
         throw new CorruptIndexException("file is too small:" + indexInput.length(), indexInput);
       }
@@ -290,7 +291,10 @@ public class S3BackupRepository implements BackupRepository {
 
         byte[] buffer = new byte[CHUNK_SIZE];
         int bufferLen;
-        long remaining = indexInput.length() - CodecUtil.footerLength();
+        long remaining =
+            shouldVerifyChecksum
+                ? indexInput.length() - CodecUtil.footerLength()
+                : indexInput.length();
 
         while (remaining > 0) {
           bufferLen = remaining >= CHUNK_SIZE ? CHUNK_SIZE : (int) remaining;
@@ -299,8 +303,10 @@ public class S3BackupRepository implements BackupRepository {
           outputStream.write(buffer, 0, bufferLen);
           remaining -= bufferLen;
         }
-        final long checksum = CodecUtil.checkFooter(indexInput);
-        writeFooter(checksum, outputStream);
+        if (shouldVerifyChecksum) {
+          long checksum = CodecUtil.checkFooter((ChecksumIndexInput) indexInput);
+          writeFooter(checksum, outputStream);
+        }
       }
     }
 
diff --git a/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepositoryConfig.java b/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepositoryConfig.java
index fcc5fc6ba08..43d6279bc94 100644
--- a/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepositoryConfig.java
+++ b/solr/modules/s3-repository/src/java/org/apache/solr/s3/S3BackupRepositoryConfig.java
@@ -57,7 +57,7 @@ public class S3BackupRepositoryConfig {
         bucketName, profile, region, proxyURL, proxyUseSystemSettings, endpoint, disableRetries);
   }
 
-  private static String getStringConfig(NamedList<?> config, String property) {
+  static String getStringConfig(NamedList<?> config, String property) {
     String envProp = EnvUtils.getProperty(property);
     if (envProp == null) {
       Object configProp = config.get(property);
@@ -67,11 +67,11 @@ public class S3BackupRepositoryConfig {
     }
   }
 
-  private static int getIntConfig(NamedList<?> config, String property) {
+  static int getIntConfig(NamedList<?> config, String property) {
     return getIntConfig(config, property, 0);
   }
 
-  private static int getIntConfig(NamedList<?> config, String property, int def) {
+  static int getIntConfig(NamedList<?> config, String property, int def) {
     String envProp = EnvUtils.getProperty(property);
     if (envProp == null) {
       Object configProp = config.get(property);
@@ -82,11 +82,11 @@ public class S3BackupRepositoryConfig {
   }
 
   /** If the property as any other value than 'true' or 'TRUE', this will default to false. */
-  private static boolean getBooleanConfig(NamedList<?> config, String property) {
+  static boolean getBooleanConfig(NamedList<?> config, String property) {
     return getBooleanConfig(config, property, false);
   }
 
-  private static boolean getBooleanConfig(NamedList<?> config, String property, boolean def) {
+  static boolean getBooleanConfig(NamedList<?> config, String property, boolean def) {
     String envProp = EnvUtils.getProperty(property);
     if (envProp == null) {
       Boolean configProp = config.getBooleanArg(property);
diff --git a/solr/modules/s3-repository/src/test/org/apache/solr/s3/S3BackupRepositoryTest.java b/solr/modules/s3-repository/src/test/org/apache/solr/s3/S3BackupRepositoryTest.java
index b7f35c574f0..71617a1f028 100644
--- a/solr/modules/s3-repository/src/test/org/apache/solr/s3/S3BackupRepositoryTest.java
+++ b/solr/modules/s3-repository/src/test/org/apache/solr/s3/S3BackupRepositoryTest.java
@@ -296,6 +296,11 @@ public class S3BackupRepositoryTest extends AbstractBackupRepositoryTest {
     }
   }
 
+  @Override
+  protected Class<? extends BackupRepository> getRepositoryClass() {
+    return S3BackupRepository.class;
+  }
+
   @Override
   protected S3BackupRepository getRepository() {
     System.setProperty("aws.accessKeyId", "foo");
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc
index 72ad3bc91ab..96ed08f8c95 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/backup-restore.adoc
@@ -388,7 +388,7 @@ If the status is anything other than "success", an error message will explain wh
 
 Solr provides a repository abstraction to allow users to backup and restore their data to a variety of different storage systems.
 For example, a Solr cluster running on a local filesystem (e.g., EXT3) can store backup data on the same disk, on a remote network-mounted drive, in HDFS, or even in some popular "cloud storage" providers, depending on the 'repository' implementation chosen.
-Solr offers multiple different repository implementations out of the box (`LocalFileSystemRepository`, `HdfsBackupRepository`, `GCSBackupRepository` and `S3BackupRepository`), and allows users to create plugins for their own storage systems as needed.
+Solr offers multiple different repository implementations out of the box (`LocalFileSystemRepository`, `HdfsBackupRepository`, `GCSBackupRepository` and `S3BackupRepository`), and allows users to create plugins for their own storage systems as needed. It is also possible to create a `DelegatingBackupRepository` that delegates to another `BackupRepository` and adds or modifies some behavior on top of it.
 
 Users can define any number of repositories in their `solr.xml` file.
 The backup and restore APIs described above allow users to select which of these definitions they want to use at runtime via the `repository` parameter.
@@ -401,6 +401,17 @@ Any children under the `<repository>` tag are passed as additional configuration
 
 Information on each of the repository implementations provided with Solr is provided below.
 
+By default, all the repository implementations verify the integrity of the index files before they are copied to the destination. However, it is possible to disable this integrity check by setting the optional configuration property `verifyChecksum`.
+
+`verifyChecksum`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: `true`
+|===
++
+Defines whether the backup repository should check the index files integrity before copying them to the destination. Set `false` to disable the checksum verification, in order to verify the integrity in a different way, for example if the files are encrypted.
+
 === LocalFileSystemRepository
 
 LocalFileSystemRepository stores and retrieves backup files anywhere on the accessible filesystem.
@@ -422,7 +433,7 @@ LocalFileSystemRepository accepts the following configuration option:
 |===
 +
 A valid file path (accessible to Solr locally) to use for backup storage and retrieval.
-Used as a fallback when user's don't provide a `location` parameter in their Backup or Restore API commands
+Used as a fallback when users don't provide a `location` parameter in their Backup or Restore API commands.
 
 An example configuration using this property can be found below.
 
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractBackupRepositoryTest.java b/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractBackupRepositoryTest.java
index 9a260d76a56..d988c5662bc 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractBackupRepositoryTest.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractBackupRepositoryTest.java
@@ -17,20 +17,47 @@
 
 package org.apache.solr.cloud.api.collections;
 
+import static org.apache.lucene.codecs.CodecUtil.FOOTER_MAGIC;
+import static org.apache.lucene.codecs.CodecUtil.writeBEInt;
+import static org.apache.lucene.codecs.CodecUtil.writeBELong;
+import static org.apache.solr.core.backup.repository.AbstractBackupRepository.PARAM_VERIFY_CHECKSUM;
+import static org.apache.solr.core.backup.repository.DelegatingBackupRepository.PARAM_DELEGATE_REPOSITORY_NAME;
+
+import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.OutputStreamIndexOutput;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
+import org.apache.solr.core.backup.repository.DelegatingBackupRepository;
+import org.apache.solr.schema.FieldType;
 import org.junit.Test;
 
 public abstract class AbstractBackupRepositoryTest extends SolrTestCaseJ4 {
+
+  protected abstract Class<? extends BackupRepository> getRepositoryClass();
+
   protected abstract BackupRepository getRepository();
 
   protected abstract URI getBaseUri() throws URISyntaxException;
@@ -62,7 +89,12 @@ public abstract class AbstractBackupRepositoryTest extends SolrTestCaseJ4 {
   @Test
   public void testCanChooseDefaultOrOverrideLocationValue() throws Exception {
     final NamedList<Object> config = getBaseBackupRepositoryConfiguration();
-    config.add(CoreAdminParams.BACKUP_LOCATION, "someLocation");
+    int locationIndex = config.indexOf(CoreAdminParams.BACKUP_LOCATION, 0);
+    if (locationIndex == -1) {
+      config.add(CoreAdminParams.BACKUP_LOCATION, "someLocation");
+    } else {
+      config.setVal(locationIndex, "someLocation");
+    }
     try (BackupRepository repo = getRepository()) {
       repo.init(config);
       assertEquals("someLocation", repo.getBackupLocation(null));
@@ -237,6 +269,121 @@ public abstract class AbstractBackupRepositoryTest extends SolrTestCaseJ4 {
     }
   }
 
+  @Test
+  public void testCanDisableChecksumVerification() throws Exception {
+    // May contain test implementation specific initialization.
+    getRepository();
+
+    // Given two BackupRepository plugins:
+    // - A standard BackupRepository plugin.
+    // - A NoChecksumFilterBackupRepository that delegates to the previous one, and adds the
+    // verifyChecksum=false parameter to the init args of the delegate.
+    String repoName = "repo";
+    String filterRepoName = "filterRepo";
+    PluginInfo[] plugins =
+        new PluginInfo[] {
+          getPluginInfo(repoName, false),
+          getNoChecksumFilterPluginInfo(filterRepoName, true, repoName),
+        };
+    Collections.shuffle(Arrays.asList(plugins), random());
+
+    // Given a file on the local disk with an invalid checksum (e.g. could be encrypted).
+    File sourceDir = createTempDir().toFile();
+    String fileName = "source-file";
+    String content = "content";
+    try (OutputStream os = FileUtils.openOutputStream(new File(sourceDir, fileName));
+        IndexOutput io = new OutputStreamIndexOutput("", "", os, Long.BYTES)) {
+      byte[] bytes = content.getBytes(StandardCharsets.UTF_8);
+      io.writeBytes(bytes, bytes.length);
+      // Instead of writing the checksum with CodecUtil.writeFooter(), write a footer with an
+      // invalid checksum.
+      writeBEInt(io, FOOTER_MAGIC);
+      writeBEInt(io, 0);
+      writeBELong(io, 1);
+    }
+
+    BackupRepositoryFactory repoFactory = new BackupRepositoryFactory(plugins);
+    try (SolrResourceLoader resourceLoader = new SolrResourceLoader(sourceDir.toPath())) {
+
+      // When we copy the local file with the standard BackupRepository,
+      // then it fails because the checksum is invalid.
+      expectThrows(
+          CorruptIndexException.class,
+          () -> copyFileToRepo(sourceDir, fileName, repoName, repoFactory, resourceLoader));
+      File destinationDir = createTempDir().toFile();
+      expectThrows(
+          CorruptIndexException.class,
+          () ->
+              copyFileToDir(
+                  sourceDir, fileName, destinationDir, repoName, repoFactory, resourceLoader));
+
+      // When we copy the local file with the NoChecksumFilterBackupRepository,
+      // then it succeeds because the checksum is not verified,
+      // and the checksum is verified alternatively.
+      NoChecksumDelegatingBackupRepository.checksumVerifiedAlternatively = false;
+      copyFileToRepo(sourceDir, fileName, filterRepoName, repoFactory, resourceLoader);
+      assertTrue(NoChecksumDelegatingBackupRepository.checksumVerifiedAlternatively);
+      NoChecksumDelegatingBackupRepository.checksumVerifiedAlternatively = false;
+      copyFileToDir(
+          sourceDir, fileName, destinationDir, filterRepoName, repoFactory, resourceLoader);
+      assertTrue(NoChecksumDelegatingBackupRepository.checksumVerifiedAlternatively);
+    }
+  }
+
+  private void copyFileToRepo(
+      File dir,
+      String fileName,
+      String repoName,
+      BackupRepositoryFactory repoFactory,
+      SolrResourceLoader resourceLoader)
+      throws IOException, URISyntaxException {
+    try (BackupRepository repo = repoFactory.newInstance(resourceLoader, repoName);
+        Directory directory = new NIOFSDirectory(dir.toPath())) {
+      URI destinationDir = repo.resolve(getBaseUri(), "destination-folder");
+      repo.copyIndexFileFrom(directory, fileName, destinationDir, fileName);
+    }
+  }
+
+  private void copyFileToDir(
+      File sourceDir,
+      String fileName,
+      File destinationDir,
+      String repoName,
+      BackupRepositoryFactory repoFactory,
+      SolrResourceLoader resourceLoader)
+      throws IOException {
+    try (BackupRepository repo = repoFactory.newInstance(resourceLoader, repoName);
+        Directory sourceDirectory = new NIOFSDirectory(sourceDir.toPath());
+        Directory destinationDirectory = new NIOFSDirectory(destinationDir.toPath())) {
+      repo.copyIndexFileFrom(sourceDirectory, fileName, destinationDirectory, fileName);
+    }
+  }
+
+  protected PluginInfo getPluginInfo(String pluginName, boolean isDefault) {
+    return getPluginInfo(pluginName, isDefault, Map.of());
+  }
+
+  protected PluginInfo getPluginInfo(
+      String pluginName, boolean isDefault, Map<String, String> attributes) {
+    Map<String, String> attrs = new HashMap<>();
+    attrs.put(CoreAdminParams.NAME, pluginName);
+    attrs.put(FieldType.CLASS_NAME, getRepositoryClass().getName());
+    attrs.put("default", Boolean.toString(isDefault));
+    attrs.putAll(attributes);
+    return new PluginInfo("repository", attrs, getBaseBackupRepositoryConfiguration(), null);
+  }
+
+  private PluginInfo getNoChecksumFilterPluginInfo(
+      String pluginName, boolean isDefault, String delegateName) {
+    Map<String, String> attrs = new HashMap<>();
+    attrs.put(CoreAdminParams.NAME, pluginName);
+    attrs.put(FieldType.CLASS_NAME, NoChecksumDelegatingBackupRepository.class.getName());
+    attrs.put("default", Boolean.toString(isDefault));
+    NamedList<Object> args = new NamedList<>();
+    args.add(PARAM_DELEGATE_REPOSITORY_NAME, delegateName);
+    return new PluginInfo("repository", attrs, args, null);
+  }
+
   private void addFile(BackupRepository repo, URI file) throws IOException {
     try (OutputStream os = repo.createOutput(file)) {
       os.write(100);
@@ -244,4 +391,49 @@ public abstract class AbstractBackupRepositoryTest extends SolrTestCaseJ4 {
       os.write(102);
     }
   }
+
+  /**
+   * Test implementation of a {@link DelegatingBackupRepository} that disables the checksum
+   * verification on its delegate {@link BackupRepository}.
+   *
+   * <p>This test class is public to be instantiated by the {@link BackupRepositoryFactory}.
+   */
+  public static class NoChecksumDelegatingBackupRepository extends DelegatingBackupRepository {
+
+    static volatile boolean checksumVerifiedAlternatively;
+
+    @Override
+    protected NamedList<?> getDelegateInitArgs(NamedList<?> initArgs) {
+      NamedList<Object> newInitArgs = new NamedList<>(initArgs.size() + 1);
+      newInitArgs.add(PARAM_VERIFY_CHECKSUM, Boolean.FALSE.toString());
+      newInitArgs.addAll(initArgs);
+      return newInitArgs;
+    }
+
+    @Override
+    public void copyIndexFileFrom(
+        Directory sourceDir, String sourceFileName, Directory destDir, String destFileName)
+        throws IOException {
+      // Verify the checksum with the original directory.
+      verifyChecksum(sourceDir, sourceFileName);
+      // Copy the index file with the unwrapped (delegate) directory.
+      super.copyIndexFileFrom(
+          FilterDirectory.unwrap(sourceDir), sourceFileName, destDir, destFileName);
+    }
+
+    @Override
+    public void copyIndexFileFrom(
+        Directory sourceDir, String sourceFileName, URI destDir, String destFileName)
+        throws IOException {
+      // Verify the checksum with the original directory.
+      verifyChecksum(sourceDir, sourceFileName);
+      // Copy the index file with the unwrapped (delegate) directory.
+      super.copyIndexFileFrom(
+          FilterDirectory.unwrap(sourceDir), sourceFileName, destDir, destFileName);
+    }
+
+    private void verifyChecksum(Directory sourceDir, String sourceFileName) {
+      checksumVerifiedAlternatively = true;
+    }
+  }
 }
diff --git a/solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java b/solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java
index 5e28d7aff62..f81d59d1d4a 100644
--- a/solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java
+++ b/solr/test-framework/src/java/org/apache/solr/core/TrackingBackupRepository.java
@@ -21,117 +21,35 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.net.URI;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.backup.Checksum;
-import org.apache.solr.core.backup.repository.BackupRepository;
-import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
+import org.apache.solr.core.backup.repository.DelegatingBackupRepository;
 
-public class TrackingBackupRepository implements BackupRepository {
+public class TrackingBackupRepository extends DelegatingBackupRepository {
   private static final List<URI> COPIED_FILES = Collections.synchronizedList(new ArrayList<>());
   private static final List<URI> DIRECTORIES_CREATED =
       Collections.synchronizedList(new ArrayList<>());
   private static final List<URI> OUTPUTS_CREATED = Collections.synchronizedList(new ArrayList<>());
 
-  private BackupRepository delegate;
-
-  @Override
-  public <T> T getConfigProperty(String name) {
-    return delegate.getConfigProperty(name);
-  }
-
-  @Override
-  public URI createURI(String path) {
-    return delegate.createURI(path);
-  }
-
-  @Override
-  public URI createDirectoryURI(String path) {
-    return delegate.createDirectoryURI(path);
-  }
-
-  @Override
-  public URI resolve(URI baseUri, String... pathComponents) {
-    return delegate.resolve(baseUri, pathComponents);
-  }
-
-  @Override
-  public URI resolveDirectory(URI baseUri, String... pathComponents) {
-    return delegate.resolveDirectory(baseUri, pathComponents);
-  }
-
-  @Override
-  public boolean exists(URI path) throws IOException {
-    return delegate.exists(path);
-  }
-
-  @Override
-  public PathType getPathType(URI path) throws IOException {
-    return delegate.getPathType(path);
-  }
-
-  @Override
-  public String[] listAll(URI path) throws IOException {
-    return delegate.listAll(path);
-  }
-
-  @Override
-  public IndexInput openInput(URI dirPath, String fileName, IOContext ctx) throws IOException {
-    return delegate.openInput(dirPath, fileName, ctx);
-  }
-
   @Override
   public OutputStream createOutput(URI path) throws IOException {
     OUTPUTS_CREATED.add(path);
-    return delegate.createOutput(path);
+    return super.createOutput(path);
   }
 
   @Override
   public void createDirectory(URI path) throws IOException {
     DIRECTORIES_CREATED.add(path);
-    delegate.createDirectory(path);
-  }
-
-  @Override
-  public void deleteDirectory(URI path) throws IOException {
-    delegate.deleteDirectory(path);
+    super.createDirectory(path);
   }
 
   @Override
   public void copyIndexFileFrom(
       Directory sourceDir, String sourceFileName, URI destDir, String destFileName)
       throws IOException {
-    COPIED_FILES.add(delegate.resolve(destDir, destFileName));
-    delegate.copyIndexFileFrom(sourceDir, sourceFileName, destDir, destFileName);
-  }
-
-  @Override
-  public void close() throws IOException {
-    delegate.close();
-  }
-
-  @Override
-  public void delete(URI path, Collection<String> files) throws IOException {
-    delegate.delete(path, files);
-  }
-
-  @Override
-  public Checksum checksum(Directory dir, String fileName) throws IOException {
-    return delegate.checksum(dir, fileName);
-  }
-
-  @Override
-  public void init(NamedList<?> args) {
-    BackupRepositoryFactory factory = (BackupRepositoryFactory) args.get("factory");
-    SolrResourceLoader loader = (SolrResourceLoader) args.get("loader");
-    String repoName = (String) args.get("delegateRepoName");
-
-    this.delegate = factory.newInstance(loader, repoName);
+    COPIED_FILES.add(resolve(destDir, destFileName));
+    super.copyIndexFileFrom(sourceDir, sourceFileName, destDir, destFileName);
   }
 
   /**
@@ -155,11 +73,4 @@ public class TrackingBackupRepository implements BackupRepository {
     DIRECTORIES_CREATED.clear();
     OUTPUTS_CREATED.clear();
   }
-
-  @Override
-  public void copyIndexFileTo(
-      URI sourceRepo, String sourceFileName, Directory dest, String destFileName)
-      throws IOException {
-    delegate.copyIndexFileTo(sourceRepo, sourceFileName, dest, destFileName);
-  }
 }