You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ge...@apache.org on 2023/05/25 11:46:02 UTC

[solr] 01/03: SOLR-16394: Tweak backup listing, deletion to be more REST-ful (#1639)

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

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

commit 7a915abc320ccf5bc4ec3f117ba6796ea845957e
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Tue May 16 12:17:27 2023 -0400

    SOLR-16394: Tweak backup listing, deletion to be more REST-ful (#1639)
    
    This commit tweaks the v2 bindings for our "delete" and "list backups" APIs to be
    more intuitive for users.  DELETEBACKUP now has two v2 bindings: one to delete
    backups by ID (`DELETE /api/backups/backupName/versions/<id>`), and one to
    delete all but the most recent 'N' backups
    (`DELETE /api/backups/backupName/versions?retainLatest=<N>`).
    
    Backup listing is now available at `GET /api/backups/backupName/versions`
---
 solr/CHANGES.txt                                   |   5 +
 .../java/org/apache/solr/core/CoreContainer.java   |   3 -
 .../apache/solr/core/backup/BackupProperties.java  |  10 +-
 .../apache/solr/handler/CollectionBackupsAPI.java  |  65 -----
 .../solr/handler/admin/CollectionsHandler.java     | 160 +----------
 .../solr/handler/admin/api/AdminAPIBase.java       |   4 +-
 .../solr/handler/admin/api/BackupAPIBase.java      | 123 +++++++++
 .../admin/api/CreateCollectionBackupAPI.java       |  54 +---
 .../admin/api/DeleteCollectionBackupAPI.java       | 299 +++++++++++++++++++++
 .../admin/api/ListCollectionBackupsAPI.java        | 149 ++++++++++
 .../handler/admin/api/RestoreCollectionAPI.java    |  22 +-
 .../admin/V2CollectionBackupsAPIMappingTest.java   |  90 -------
 .../admin/api/DeleteCollectionBackupAPITest.java   | 154 +++++++++++
 .../pages/collection-management.adoc               | 100 +++++--
 .../solrj/request/CollectionAdminRequest.java      |  53 ++--
 .../solrj/request/beans/DeleteBackupPayload.java   |  38 ---
 .../solrj/request/beans/ListBackupPayload.java     |  29 --
 .../collections/AbstractIncrementalBackupTest.java |   5 +-
 18 files changed, 858 insertions(+), 505 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 4582096ca0c..4c385d3271d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -99,6 +99,11 @@ Improvements
 * SOLR-9378: Internal shard requests no longer include the wasteful shard.url param.  [shard] transformer now defaults to returning 
   only the shard id (based on luceneMatchVersion), but can be configured to return the legacy list of replicas. (hossman)
 
+* SOLR-16394: The v2 list and delete (collection) backup APIs have been tweaked to be more intuitive: backup listing now uses
+  `GET /api/backups/bName/versions`, backup deletion by ID now uses `DELETE /api/backups/bName/versions/1`, backup deletion by
+  recency now uses `DELETE /api/backups/bName/versions?retainLatest=3`, and index-file "garbage collection" now uses
+  `PUT /api/backups/backupName/purgeUnused` (Jason Gerlowski)
+
 Optimizations
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index edde8c34cf3..e4f72587142 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -108,7 +108,6 @@ import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
 import org.apache.solr.filestore.PackageStoreAPI;
 import org.apache.solr.handler.ClusterAPI;
-import org.apache.solr.handler.CollectionBackupsAPI;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.CollectionsHandler;
@@ -852,8 +851,6 @@ public class CoreContainer {
     collectionsHandler =
         createHandler(
             COLLECTIONS_HANDLER_PATH, cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
-    final CollectionBackupsAPI collectionBackupsAPI = new CollectionBackupsAPI(collectionsHandler);
-    registerV2ApiIfEnabled(collectionBackupsAPI);
     configSetsHandler =
         createHandler(
             CONFIGSETS_HANDLER_PATH, cfg.getConfigSetsHandlerClass(), ConfigSetsHandler.class);
diff --git a/solr/core/src/java/org/apache/solr/core/backup/BackupProperties.java b/solr/core/src/java/org/apache/solr/core/backup/BackupProperties.java
index 741afe760d6..1e452bb516d 100644
--- a/solr/core/src/java/org/apache/solr/core/backup/BackupProperties.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/BackupProperties.java
@@ -158,17 +158,19 @@ public class BackupProperties {
     return properties.getProperty(BackupManager.INDEX_VERSION_PROP);
   }
 
-  public Map<Object, Object> getDetails() {
-    Map<Object, Object> result = new HashMap<>(properties);
+  public Map<String, Object> getDetails() {
+    final Map<String, Object> result = new HashMap<>();
+    properties.entrySet().stream()
+        .forEach(entry -> result.put(entry.getKey().toString(), entry.getValue()));
     result.remove(BackupManager.BACKUP_NAME_PROP);
     result.remove(BackupManager.COLLECTION_NAME_PROP);
     result.put("indexSizeMB", Double.valueOf(properties.getProperty("indexSizeMB")));
     result.put("indexFileCount", Integer.valueOf(properties.getProperty("indexFileCount")));
 
     Map<String, String> shardBackupIds = new HashMap<>();
-    Iterator<Object> keyIt = result.keySet().iterator();
+    Iterator<String> keyIt = result.keySet().iterator();
     while (keyIt.hasNext()) {
-      String key = keyIt.next().toString();
+      String key = keyIt.next();
       if (key.endsWith(".md")) {
         shardBackupIds.put(key.substring(0, key.length() - 3), properties.getProperty(key));
         keyIt.remove();
diff --git a/solr/core/src/java/org/apache/solr/handler/CollectionBackupsAPI.java b/solr/core/src/java/org/apache/solr/handler/CollectionBackupsAPI.java
deleted file mode 100644
index 4ff7693b371..00000000000
--- a/solr/core/src/java/org/apache/solr/handler/CollectionBackupsAPI.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.handler;
-
-import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
-import static org.apache.solr.common.params.CommonParams.ACTION;
-import static org.apache.solr.handler.ClusterAPI.wrapParams;
-import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.solr.api.Command;
-import org.apache.solr.api.EndPoint;
-import org.apache.solr.api.PayloadObj;
-import org.apache.solr.client.solrj.request.beans.DeleteBackupPayload;
-import org.apache.solr.client.solrj.request.beans.ListBackupPayload;
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.handler.admin.CollectionsHandler;
-
-/** V2 API definitions for */
-@EndPoint(
-    path = {"/c/backups", "/collections/backups"},
-    method = POST,
-    permission = COLL_EDIT_PERM)
-public class CollectionBackupsAPI {
-
-  public static final String LIST_BACKUP_CMD = "list-backups";
-  public static final String DELETE_BACKUP_CMD = "delete-backups";
-
-  private final CollectionsHandler collectionsHandler;
-
-  public CollectionBackupsAPI(CollectionsHandler collectionsHandler) {
-    this.collectionsHandler = collectionsHandler;
-  }
-
-  @Command(name = LIST_BACKUP_CMD)
-  public void listBackups(PayloadObj<ListBackupPayload> obj) throws Exception {
-    final Map<String, Object> v1Params = obj.get().toMap(new HashMap<>());
-    v1Params.put(ACTION, CollectionParams.CollectionAction.LISTBACKUP.toLower());
-
-    collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
-  }
-
-  @Command(name = DELETE_BACKUP_CMD)
-  public void deleteBackups(PayloadObj<DeleteBackupPayload> obj) throws Exception {
-    final Map<String, Object> v1Params = obj.get().toMap(new HashMap<>());
-    v1Params.put(ACTION, CollectionParams.CollectionAction.DELETEBACKUP.toLower());
-
-    collectionsHandler.handleRequestBody(wrapParams(obj.getRequest(), v1Params), obj.getResponse());
-  }
-}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 499e58b4c3a..f222fa2e801 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -104,20 +104,15 @@ import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STA
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.CommonParams.TIMING;
 import static org.apache.solr.common.params.CommonParams.VALUE_LONG;
-import static org.apache.solr.common.params.CoreAdminParams.BACKUP_ID;
 import static org.apache.solr.common.params.CoreAdminParams.BACKUP_LOCATION;
-import static org.apache.solr.common.params.CoreAdminParams.BACKUP_PURGE_UNUSED;
 import static org.apache.solr.common.params.CoreAdminParams.BACKUP_REPOSITORY;
 import static org.apache.solr.common.params.CoreAdminParams.DATA_DIR;
 import static org.apache.solr.common.params.CoreAdminParams.INSTANCE_DIR;
-import static org.apache.solr.common.params.CoreAdminParams.MAX_NUM_BACKUP_POINTS;
 import static org.apache.solr.common.params.CoreAdminParams.ULOG_DIR;
 import static org.apache.solr.common.params.ShardParams._ROUTE_;
 import static org.apache.solr.common.util.StrUtils.formatString;
 
-import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.net.URI;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -180,11 +175,6 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CloudConfig;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.backup.BackupFilePaths;
-import org.apache.solr.core.backup.BackupId;
-import org.apache.solr.core.backup.BackupManager;
-import org.apache.solr.core.backup.BackupProperties;
-import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.snapshots.CollectionSnapshotMetaData;
 import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.handler.RequestHandlerBase;
@@ -202,6 +192,7 @@ import org.apache.solr.handler.admin.api.CreateCollectionSnapshotAPI;
 import org.apache.solr.handler.admin.api.CreateShardAPI;
 import org.apache.solr.handler.admin.api.DeleteAliasAPI;
 import org.apache.solr.handler.admin.api.DeleteCollectionAPI;
+import org.apache.solr.handler.admin.api.DeleteCollectionBackupAPI;
 import org.apache.solr.handler.admin.api.DeleteCollectionSnapshotAPI;
 import org.apache.solr.handler.admin.api.DeleteNodeAPI;
 import org.apache.solr.handler.admin.api.DeleteReplicaAPI;
@@ -210,6 +201,7 @@ import org.apache.solr.handler.admin.api.DeleteShardAPI;
 import org.apache.solr.handler.admin.api.ForceLeaderAPI;
 import org.apache.solr.handler.admin.api.InstallShardDataAPI;
 import org.apache.solr.handler.admin.api.ListAliasesAPI;
+import org.apache.solr.handler.admin.api.ListCollectionBackupsAPI;
 import org.apache.solr.handler.admin.api.ListCollectionSnapshotsAPI;
 import org.apache.solr.handler.admin.api.ListCollectionsAPI;
 import org.apache.solr.handler.admin.api.MigrateDocsAPI;
@@ -1205,153 +1197,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     DELETEBACKUP_OP(
         DELETEBACKUP,
         (req, rsp, h) -> {
-          req.getParams().required().check(NAME);
-
-          CoreContainer cc = h.coreContainer;
-          String repo = req.getParams().get(CoreAdminParams.BACKUP_REPOSITORY);
-          try (BackupRepository repository = cc.newBackupRepository(repo)) {
-
-            String location =
-                repository.getBackupLocation(req.getParams().get(CoreAdminParams.BACKUP_LOCATION));
-            if (location == null) {
-              // Refresh the cluster property file to make sure the value set for location is the
-              // latest. Check if the location is specified in the cluster property.
-              location =
-                  new ClusterProperties(h.coreContainer.getZkController().getZkClient())
-                      .getClusterProperty("location", null);
-              if (location == null) {
-                throw new SolrException(
-                    ErrorCode.BAD_REQUEST,
-                    "'location' is not specified as a query"
-                        + " parameter or as a default repository property or as a cluster property.");
-              }
-            }
-
-            // Check if the specified location is valid for this repository.
-            URI uri = repository.createDirectoryURI(location);
-            try {
-              if (!repository.exists(uri)) {
-                throw new SolrException(
-                    ErrorCode.BAD_REQUEST, "specified location " + uri + " does not exist.");
-              }
-            } catch (IOException ex) {
-              throw new SolrException(
-                  ErrorCode.SERVER_ERROR,
-                  "Failed to check the existence of " + uri + ". Is it valid?",
-                  ex);
-            }
-
-            int deletionModesProvided = 0;
-            if (req.getParams().get(MAX_NUM_BACKUP_POINTS) != null) deletionModesProvided++;
-            if (req.getParams().get(BACKUP_PURGE_UNUSED) != null) deletionModesProvided++;
-            if (req.getParams().get(BACKUP_ID) != null) deletionModesProvided++;
-            if (deletionModesProvided != 1) {
-              throw new SolrException(
-                  BAD_REQUEST,
-                  String.format(
-                      Locale.ROOT,
-                      "Exactly one of %s, %s, and %s parameters must be provided",
-                      MAX_NUM_BACKUP_POINTS,
-                      BACKUP_PURGE_UNUSED,
-                      BACKUP_ID));
-            }
-
-            final Map<String, Object> params =
-                copy(
-                    req.getParams(),
-                    null,
-                    NAME,
-                    BACKUP_REPOSITORY,
-                    BACKUP_LOCATION,
-                    BACKUP_ID,
-                    MAX_NUM_BACKUP_POINTS,
-                    BACKUP_PURGE_UNUSED);
-            params.put(BACKUP_LOCATION, location);
-            if (repo != null) {
-              params.put(CoreAdminParams.BACKUP_REPOSITORY, repo);
-            }
-            return params;
-          }
+          DeleteCollectionBackupAPI.invokeFromV1Params(h.coreContainer, req, rsp);
+          return null;
         }),
     LISTBACKUP_OP(
         LISTBACKUP,
         (req, rsp, h) -> {
           req.getParams().required().check(NAME);
-
-          CoreContainer cc = h.coreContainer;
-          String repo = req.getParams().get(CoreAdminParams.BACKUP_REPOSITORY);
-          try (BackupRepository repository = cc.newBackupRepository(repo)) {
-
-            String location =
-                repository.getBackupLocation(req.getParams().get(CoreAdminParams.BACKUP_LOCATION));
-            if (location == null) {
-              // Refresh the cluster property file to make sure the value set for location is the
-              // latest. Check if the location is specified in the cluster property.
-              location =
-                  new ClusterProperties(h.coreContainer.getZkController().getZkClient())
-                      .getClusterProperty(CoreAdminParams.BACKUP_LOCATION, null);
-              if (location == null) {
-                throw new SolrException(
-                    ErrorCode.BAD_REQUEST,
-                    "'location' is not specified as a query"
-                        + " parameter or as a default repository property or as a cluster property.");
-              }
-            }
-
-            String backupName = req.getParams().get(NAME);
-            final URI locationURI = repository.createDirectoryURI(location);
-            try {
-              if (!repository.exists(locationURI)) {
-                throw new SolrException(
-                    ErrorCode.BAD_REQUEST,
-                    "specified location " + locationURI + " does not exist.");
-              }
-            } catch (IOException ex) {
-              throw new SolrException(
-                  ErrorCode.SERVER_ERROR,
-                  "Failed to check the existence of " + locationURI + ". Is it valid?",
-                  ex);
-            }
-            URI backupLocation =
-                BackupFilePaths.buildExistingBackupLocationURI(repository, locationURI, backupName);
-            if (repository.exists(
-                repository.resolve(backupLocation, BackupManager.TRADITIONAL_BACKUP_PROPS_FILE))) {
-              throw new SolrException(
-                  SolrException.ErrorCode.BAD_REQUEST,
-                  "The backup name ["
-                      + backupName
-                      + "] at "
-                      + "location ["
-                      + location
-                      + "] holds a non-incremental (legacy) backup, but "
-                      + "backup-listing is only supported on incremental backups");
-            }
-
-            String[] subFiles = repository.listAllOrEmpty(backupLocation);
-            List<BackupId> propsFiles = BackupFilePaths.findAllBackupIdsFromFileListing(subFiles);
-
-            NamedList<Object> results = new NamedList<>();
-            ArrayList<Map<Object, Object>> backups = new ArrayList<>();
-            String collectionName = null;
-            for (BackupId backupId : propsFiles) {
-              BackupProperties properties =
-                  BackupProperties.readFrom(
-                      repository, backupLocation, BackupFilePaths.getBackupPropsName(backupId));
-              if (collectionName == null) {
-                collectionName = properties.getCollection();
-                results.add(BackupManager.COLLECTION_NAME_PROP, collectionName);
-              }
-
-              Map<Object, Object> details = properties.getDetails();
-              details.put("backupId", backupId.id);
-              backups.add(details);
-            }
-
-            results.add("backups", backups);
-            SolrResponse response = new OverseerSolrResponse(results);
-            rsp.getValues().addAll(response.getResponse());
-            return null;
-          }
+          ListCollectionBackupsAPI.invokeFromV1Params(h.coreContainer, req, rsp);
+          return null;
         }),
     CREATESNAPSHOT_OP(
         CREATESNAPSHOT,
@@ -1705,12 +1559,14 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
         CreateCollectionAPI.class,
         CreateCollectionBackupAPI.class,
         DeleteAliasAPI.class,
+        DeleteCollectionBackupAPI.class,
         DeleteCollectionAPI.class,
         DeleteReplicaAPI.class,
         DeleteReplicaPropertyAPI.class,
         DeleteShardAPI.class,
         InstallShardDataAPI.class,
         ListCollectionsAPI.class,
+        ListCollectionBackupsAPI.class,
         ReplaceNodeAPI.class,
         RestoreCollectionAPI.class,
         CollectionPropertyAPI.class,
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/AdminAPIBase.java b/solr/core/src/java/org/apache/solr/handler/admin/api/AdminAPIBase.java
index ab2610afd74..17890115d2b 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/AdminAPIBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/AdminAPIBase.java
@@ -124,7 +124,7 @@ public abstract class AdminAPIBase extends JerseyResource {
     solrQueryResponse.setHttpCaching(false);
   }
 
-  protected SubResponseAccumulatingJerseyResponse submitRemoteMessageAndHandleResponse(
+  protected SolrResponse submitRemoteMessageAndHandleResponse(
       SubResponseAccumulatingJerseyResponse response,
       CollectionParams.CollectionAction action,
       ZkNodeProps remoteMessage,
@@ -149,7 +149,7 @@ public abstract class AdminAPIBase extends JerseyResource {
     response.successfulSubResponsesByNodeName = remoteResponse.getResponse().get("success");
     response.failedSubResponsesByNodeName = remoteResponse.getResponse().get("failure");
 
-    return response;
+    return remoteResponse;
   }
 
   protected static void insertIfNotNull(Map<String, Object> destination, String key, Object value) {
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/BackupAPIBase.java b/solr/core/src/java/org/apache/solr/handler/admin/api/BackupAPIBase.java
new file mode 100644
index 00000000000..b7028ff0405
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/BackupAPIBase.java
@@ -0,0 +1,123 @@
+/*
+ * 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.handler.admin.api;
+
+import java.io.IOException;
+import java.net.URI;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterProperties;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.backup.BackupFilePaths;
+import org.apache.solr.core.backup.BackupManager;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/** Base class that facilitates reuse of common validation logic for collection-backup APIs. */
+public abstract class BackupAPIBase extends AdminAPIBase {
+
+  public BackupAPIBase(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
+
+  protected String getAndValidateBackupLocation(String repositoryName, String location)
+      throws IOException {
+    try (final var repository = createBackupRepository(repositoryName)) {
+      return getAndValidateBackupLocation(repository, location);
+    }
+  }
+
+  protected String getAndValidateIncrementalBackupLocation(
+      String repositoryName, String location, String backupName) throws IOException {
+    try (final var repository = createBackupRepository(repositoryName)) {
+      final String fetchedLocation = getAndValidateBackupLocation(repository, location);
+      ensureBackupLocationIsIncremental(repository, fetchedLocation, backupName);
+      return fetchedLocation;
+    }
+  }
+
+  protected BackupRepository createBackupRepository(String repositoryName) {
+    return coreContainer.newBackupRepository(repositoryName);
+  }
+
+  private String getLocation(BackupRepository repository, String location) throws IOException {
+    location = repository.getBackupLocation(location);
+    if (location != null) {
+      return location;
+    }
+
+    // Refresh the cluster property file to make sure the value set for location is the
+    // latest. Check if the location is specified in the cluster property.
+    location =
+        new ClusterProperties(coreContainer.getZkController().getZkClient())
+            .getClusterProperty(CoreAdminParams.BACKUP_LOCATION, null);
+    if (location != null) {
+      return location;
+    }
+
+    throw new SolrException(
+        SolrException.ErrorCode.BAD_REQUEST,
+        "'location' is not specified as a query"
+            + " parameter or as a default repository property or as a cluster property.");
+  }
+
+  private void ensureBackupLocationExists(BackupRepository repository, String location) {
+    final URI uri = repository.createDirectoryURI(location);
+    try {
+      if (!repository.exists(uri)) {
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR, "specified location " + uri + " does not exist.");
+      }
+    } catch (IOException ex) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Failed to check the existence of " + uri + ". Is it valid?",
+          ex);
+    }
+  }
+
+  private String getAndValidateBackupLocation(BackupRepository backupRepository, String location)
+      throws IOException {
+    location = getLocation(backupRepository, location);
+    ensureBackupLocationExists(backupRepository, location);
+    return location;
+  }
+
+  private void ensureBackupLocationIsIncremental(
+      BackupRepository repository, String location, String backupName) throws IOException {
+    final URI locationURI = repository.createDirectoryURI(location);
+    final var locationAndNameUri =
+        BackupFilePaths.buildExistingBackupLocationURI(repository, locationURI, backupName);
+    if (repository.exists(
+        repository.resolve(locationAndNameUri, BackupManager.TRADITIONAL_BACKUP_PROPS_FILE))) {
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST,
+          "The backup name ["
+              + backupName
+              + "] at "
+              + "location ["
+              + location
+              + "] holds a non-incremental (legacy) backup, but "
+              + "backup-listing is only supported on incremental backups");
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollectionBackupAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollectionBackupAPI.java
index 52e6108183e..01cb621ac4b 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollectionBackupAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollectionBackupAPI.java
@@ -34,8 +34,6 @@ import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PER
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import java.io.IOException;
-import java.net.URI;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -46,14 +44,11 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.jersey.JacksonReflectMapWriter;
 import org.apache.solr.jersey.PermissionName;
@@ -70,7 +65,7 @@ import org.apache.zookeeper.common.StringUtils;
  * <p>This API is analogous to the v1 /admin/collections?action=BACKUP command.
  */
 @Path("/collections/{collectionName}/backups/{backupName}/versions")
-public class CreateCollectionBackupAPI extends AdminAPIBase {
+public class CreateCollectionBackupAPI extends BackupAPIBase {
   private final ObjectMapper objectMapper;
 
   @Inject
@@ -109,26 +104,12 @@ public class CreateCollectionBackupAPI extends AdminAPIBase {
         resolveAndValidateAliasIfEnabled(
             collectionName, Boolean.TRUE.equals(requestBody.followAliases));
 
-    final BackupRepository repository = coreContainer.newBackupRepository(requestBody.repository);
-    requestBody.location = getLocation(coreContainer, repository, requestBody.location);
+    requestBody.location =
+        getAndValidateBackupLocation(requestBody.repository, requestBody.location);
+
     if (requestBody.incremental == null) {
       requestBody.incremental = Boolean.TRUE;
     }
-
-    // Check if the specified location is valid for this repository.
-    final URI uri = repository.createDirectoryURI(requestBody.location);
-    try {
-      if (!repository.exists(uri)) {
-        throw new SolrException(
-            SolrException.ErrorCode.SERVER_ERROR, "specified location " + uri + " does not exist.");
-      }
-    } catch (IOException ex) {
-      throw new SolrException(
-          SolrException.ErrorCode.SERVER_ERROR,
-          "Failed to check the existence of " + uri + ". Is it valid?",
-          ex);
-    }
-
     if (requestBody.backupStrategy == null) {
       requestBody.backupStrategy = CollectionAdminParams.COPY_FILES_STRATEGY;
     }
@@ -198,29 +179,6 @@ public class CreateCollectionBackupAPI extends AdminAPIBase {
     return createBackupApi.createCollectionBackup(collectionName, backupName, requestBody);
   }
 
-  public static String getLocation(
-      CoreContainer coreContainer, BackupRepository repository, String location)
-      throws IOException {
-    location = repository.getBackupLocation(location);
-    if (location != null) {
-      return location;
-    }
-
-    // Refresh the cluster property file to make sure the value set for location is the
-    // latest. Check if the location is specified in the cluster property.
-    location =
-        new ClusterProperties(coreContainer.getZkController().getZkClient())
-            .getClusterProperty(CoreAdminParams.BACKUP_LOCATION, null);
-    if (location != null) {
-      return location;
-    }
-
-    throw new SolrException(
-        SolrException.ErrorCode.BAD_REQUEST,
-        "'location' is not specified as a query"
-            + " parameter or as a default repository property or as a cluster property.");
-  }
-
   public static class CreateCollectionBackupRequestBody implements JacksonReflectMapWriter {
     @JsonProperty public String location;
     @JsonProperty public String repository;
@@ -235,7 +193,7 @@ public class CreateCollectionBackupAPI extends AdminAPIBase {
   public static class CreateCollectionBackupResponseBody
       extends SubResponseAccumulatingJerseyResponse {
     @JsonProperty("response")
-    public CollectionBackupData backupDataResponse;
+    public CollectionBackupDetails backupDataResponse;
 
     @JsonProperty("deleted")
     public List<BackupDeletionData> deleted;
@@ -243,7 +201,7 @@ public class CreateCollectionBackupAPI extends AdminAPIBase {
     @JsonProperty public String collection;
   }
 
-  public static class CollectionBackupData implements JacksonReflectMapWriter {
+  public static class CollectionBackupDetails implements JacksonReflectMapWriter {
     @JsonProperty public String collection;
     @JsonProperty public Integer numShards;
     @JsonProperty public Integer backupId;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/DeleteCollectionBackupAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/DeleteCollectionBackupAPI.java
new file mode 100644
index 00000000000..4edc0ec9c6f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/DeleteCollectionBackupAPI.java
@@ -0,0 +1,299 @@
+/*
+ * 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.handler.admin.api;
+
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_ID;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_LOCATION;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_PURGE_UNUSED;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_REPOSITORY;
+import static org.apache.solr.common.params.CoreAdminParams.COLLECTION;
+import static org.apache.solr.common.params.CoreAdminParams.MAX_NUM_BACKUP_POINTS;
+import static org.apache.solr.common.params.CoreAdminParams.NAME;
+import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import javax.inject.Inject;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJacksonMapper;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.jersey.SubResponseAccumulatingJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/**
+ * V2 API definitions for collection-backup deletion.
+ *
+ * <p>These APIs are equivalent to the v1 '/admin/collections?action=DELETEBACKUP' command.
+ */
+public class DeleteCollectionBackupAPI extends BackupAPIBase {
+
+  private final ObjectMapper objectMapper;
+
+  @Inject
+  public DeleteCollectionBackupAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+
+    this.objectMapper = SolrJacksonMapper.getObjectMapper();
+  }
+
+  @Path("/backups/{backupName}/versions/{backupId}")
+  @DELETE
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public BackupDeletionResponseBody deleteSingleBackupById(
+      @PathParam("backupName") String backupName,
+      @PathParam(BACKUP_ID) String backupId,
+      // Optional parameters below
+      @QueryParam(BACKUP_LOCATION) String location,
+      @QueryParam(BACKUP_REPOSITORY) String repositoryName,
+      @QueryParam(ASYNC) String asyncId)
+      throws Exception {
+    final var response = instantiateJerseyResponse(BackupDeletionResponseBody.class);
+    recordCollectionForLogAndTracing(null, solrQueryRequest);
+
+    ensureRequiredParameterProvided(NAME, backupName);
+    ensureRequiredParameterProvided(BACKUP_ID, backupId);
+    location = getAndValidateBackupLocation(repositoryName, location);
+
+    final ZkNodeProps remoteMessage =
+        createRemoteMessage(backupName, backupId, null, null, location, repositoryName, asyncId);
+    final var remoteResponse =
+        submitRemoteMessageAndHandleResponse(
+            response, CollectionParams.CollectionAction.DELETEBACKUP, remoteMessage, asyncId);
+    response.deleted = fromRemoteResponse(objectMapper, remoteResponse);
+    response.collection = (String) remoteResponse.getResponse().get(COLLECTION);
+    return response;
+  }
+
+  @Path("/backups/{backupName}/versions")
+  @DELETE
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public BackupDeletionResponseBody deleteMultipleBackupsByRecency(
+      @PathParam("backupName") String backupName,
+      @QueryParam("retainLatest") Integer versionsToRetain,
+      // Optional parameters below
+      @QueryParam(BACKUP_LOCATION) String location,
+      @QueryParam(BACKUP_REPOSITORY) String repositoryName,
+      @QueryParam(ASYNC) String asyncId)
+      throws Exception {
+    final var response = instantiateJerseyResponse(BackupDeletionResponseBody.class);
+    recordCollectionForLogAndTracing(null, solrQueryRequest);
+
+    ensureRequiredParameterProvided(NAME, backupName);
+    ensureRequiredParameterProvided("retainLatest", versionsToRetain);
+    location = getAndValidateBackupLocation(repositoryName, location);
+
+    final ZkNodeProps remoteMessage =
+        createRemoteMessage(
+            backupName, null, versionsToRetain, null, location, repositoryName, asyncId);
+    final var remoteResponse =
+        submitRemoteMessageAndHandleResponse(
+            response, CollectionParams.CollectionAction.DELETEBACKUP, remoteMessage, asyncId);
+    response.deleted = fromRemoteResponse(objectMapper, remoteResponse);
+    response.collection = (String) remoteResponse.getResponse().get(COLLECTION);
+    return response;
+  }
+
+  @Path("/backups/{backupName}/purgeUnused")
+  @PUT
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public PurgeUnusedResponse garbageCollectUnusedBackupFiles(
+      @PathParam("backupName") String backupName, PurgeUnusedFilesRequestBody requestBody)
+      throws Exception {
+    final var response = instantiateJerseyResponse(PurgeUnusedResponse.class);
+    recordCollectionForLogAndTracing(null, solrQueryRequest);
+
+    if (requestBody == null) {
+      throw new SolrException(BAD_REQUEST, "Required request body is missing");
+    }
+    ensureRequiredParameterProvided(NAME, backupName);
+    requestBody.location =
+        getAndValidateBackupLocation(requestBody.repositoryName, requestBody.location);
+
+    final ZkNodeProps remoteMessage =
+        createRemoteMessage(
+            backupName,
+            null,
+            null,
+            Boolean.TRUE,
+            requestBody.location,
+            requestBody.repositoryName,
+            requestBody.asyncId);
+    final var remoteResponse =
+        submitRemoteMessageAndHandleResponse(
+            response,
+            CollectionParams.CollectionAction.DELETEBACKUP,
+            remoteMessage,
+            requestBody.asyncId);
+
+    final Object remoteDeleted = remoteResponse.getResponse().get("deleted");
+    if (remoteDeleted != null) {
+      response.deleted = objectMapper.convertValue(remoteDeleted, PurgeUnusedStats.class);
+    }
+    return response;
+  }
+
+  public static class PurgeUnusedResponse extends SubResponseAccumulatingJerseyResponse {
+    @JsonProperty public PurgeUnusedStats deleted;
+  }
+
+  public static class PurgeUnusedStats implements JacksonReflectMapWriter {
+    @JsonProperty public Integer numBackupIds;
+    @JsonProperty public Integer numShardBackupIds;
+    @JsonProperty public Integer numIndexFiles;
+  }
+
+  public static ZkNodeProps createRemoteMessage(
+      String backupName,
+      String backupId,
+      Integer versionsToRetain,
+      Boolean purgeUnused,
+      String location,
+      String repositoryName,
+      String asyncId) {
+    final Map<String, Object> remoteMessage = new HashMap<>();
+
+    // Always provided
+    remoteMessage.put(QUEUE_OPERATION, CollectionParams.CollectionAction.DELETEBACKUP.toLower());
+    remoteMessage.put(NAME, backupName);
+    // Mutually exclusive
+    assert backupId != null || versionsToRetain != null || purgeUnused != null;
+    insertIfNotNull(remoteMessage, BACKUP_ID, backupId);
+    insertIfNotNull(remoteMessage, MAX_NUM_BACKUP_POINTS, versionsToRetain);
+    insertIfNotNull(remoteMessage, BACKUP_PURGE_UNUSED, purgeUnused);
+    // Remaining params are truly optional
+    insertIfNotNull(remoteMessage, BACKUP_LOCATION, location);
+    insertIfNotNull(remoteMessage, BACKUP_REPOSITORY, repositoryName);
+    insertIfNotNull(remoteMessage, ASYNC, asyncId);
+
+    return new ZkNodeProps(remoteMessage);
+  }
+
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer, SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    int deletionModesProvided = 0;
+    if (req.getParams().get(MAX_NUM_BACKUP_POINTS) != null) deletionModesProvided++;
+    if (req.getParams().get(BACKUP_PURGE_UNUSED) != null) deletionModesProvided++;
+    if (req.getParams().get(BACKUP_ID) != null) deletionModesProvided++;
+    if (deletionModesProvided != 1) {
+      throw new SolrException(
+          BAD_REQUEST,
+          String.format(
+              Locale.ROOT,
+              "Exactly one of %s, %s, and %s parameters must be provided",
+              MAX_NUM_BACKUP_POINTS,
+              BACKUP_PURGE_UNUSED,
+              BACKUP_ID));
+    }
+
+    final var deleteApi = new DeleteCollectionBackupAPI(coreContainer, req, rsp);
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(rsp, invokeApi(deleteApi, req.getParams()));
+  }
+
+  public static class BackupDeletionResponseBody extends SubResponseAccumulatingJerseyResponse {
+    @JsonProperty public String collection;
+    @JsonProperty public List<CreateCollectionBackupAPI.BackupDeletionData> deleted;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static List<CreateCollectionBackupAPI.BackupDeletionData> fromRemoteResponse(
+      ObjectMapper objectMapper, SolrResponse response) {
+    final var deleted = (List<SimpleOrderedMap<Object>>) response.getResponse().get("deleted");
+    if (deleted == null) {
+      return null;
+    }
+
+    final List<CreateCollectionBackupAPI.BackupDeletionData> statList = new ArrayList<>();
+    for (SimpleOrderedMap<Object> remoteStat : deleted) {
+      statList.add(
+          objectMapper.convertValue(
+              remoteStat, CreateCollectionBackupAPI.BackupDeletionData.class));
+    }
+    return statList;
+  }
+
+  /**
+   * Request body for the {@link DeleteCollectionBackupAPI#garbageCollectUnusedBackupFiles(String,
+   * PurgeUnusedFilesRequestBody)} API.
+   */
+  public static class PurgeUnusedFilesRequestBody implements JacksonReflectMapWriter {
+    @JsonProperty(BACKUP_LOCATION)
+    public String location;
+
+    @JsonProperty(BACKUP_REPOSITORY)
+    public String repositoryName;
+
+    @JsonProperty(ASYNC)
+    public String asyncId;
+  }
+
+  private static SolrJerseyResponse invokeApi(DeleteCollectionBackupAPI api, SolrParams params)
+      throws Exception {
+    if (params.get(MAX_NUM_BACKUP_POINTS) != null) {
+      return api.deleteMultipleBackupsByRecency(
+          params.get(NAME),
+          params.getInt(MAX_NUM_BACKUP_POINTS),
+          params.get(BACKUP_LOCATION),
+          params.get(BACKUP_REPOSITORY),
+          params.get(ASYNC));
+    } else if (params.get(BACKUP_PURGE_UNUSED) != null) {
+      final var requestBody = new PurgeUnusedFilesRequestBody();
+      requestBody.location = params.get(BACKUP_LOCATION);
+      requestBody.repositoryName = params.get(BACKUP_REPOSITORY);
+      requestBody.asyncId = params.get(ASYNC);
+      return api.garbageCollectUnusedBackupFiles(params.get(NAME), requestBody);
+    } else { // BACKUP_ID != null
+      return api.deleteSingleBackupById(
+          params.get(NAME),
+          params.get(BACKUP_ID),
+          params.get(BACKUP_LOCATION),
+          params.get(BACKUP_REPOSITORY),
+          params.get(ASYNC));
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/ListCollectionBackupsAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/ListCollectionBackupsAPI.java
new file mode 100644
index 00000000000..89827b9ac03
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/ListCollectionBackupsAPI.java
@@ -0,0 +1,149 @@
+/*
+ * 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.handler.admin.api;
+
+import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_ID;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_LOCATION;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_REPOSITORY;
+import static org.apache.solr.common.params.CoreAdminParams.NAME;
+import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.backup.BackupFilePaths;
+import org.apache.solr.core.backup.BackupId;
+import org.apache.solr.core.backup.BackupProperties;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJacksonMapper;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/**
+ * V2 API definitions for collection-backup "listing".
+ *
+ * <p>These APIs are equivalent to the v1 '/admin/collections?action=LISTBACKUP' command.
+ */
+public class ListCollectionBackupsAPI extends BackupAPIBase {
+
+  private final ObjectMapper objectMapper;
+
+  @Inject
+  public ListCollectionBackupsAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+
+    this.objectMapper = SolrJacksonMapper.getObjectMapper();
+  }
+
+  public static void invokeFromV1Params(
+      CoreContainer coreContainer, SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    final SolrParams v1Params = req.getParams();
+    v1Params.required().check(CommonParams.NAME);
+
+    final var listApi = new ListCollectionBackupsAPI(coreContainer, req, rsp);
+    V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+        rsp,
+        listApi.listBackupsAtLocation(
+            v1Params.get(NAME), v1Params.get(BACKUP_LOCATION), v1Params.get(BACKUP_REPOSITORY)));
+  }
+
+  @Path("/backups/{backupName}/versions")
+  @GET
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  public ListCollectionBackupsResponse listBackupsAtLocation(
+      @PathParam("backupName") String backupName,
+      @QueryParam(BACKUP_LOCATION) String location,
+      @QueryParam(BACKUP_REPOSITORY) String repositoryName)
+      throws IOException {
+    final var response = instantiateJerseyResponse(ListCollectionBackupsResponse.class);
+    recordCollectionForLogAndTracing(null, solrQueryRequest);
+
+    ensureRequiredParameterProvided(NAME, backupName);
+    location = getAndValidateIncrementalBackupLocation(repositoryName, location, backupName);
+
+    try (final var repository = createBackupRepository(repositoryName)) {
+      final URI locationURI = repository.createDirectoryURI(location);
+      final var backupLocation =
+          BackupFilePaths.buildExistingBackupLocationURI(repository, locationURI, backupName);
+
+      String[] subFiles = repository.listAllOrEmpty(backupLocation);
+      List<BackupId> propsFiles = BackupFilePaths.findAllBackupIdsFromFileListing(subFiles);
+
+      response.backups = new ArrayList<>();
+      for (BackupId backupId : propsFiles) {
+        BackupProperties properties =
+            BackupProperties.readFrom(
+                repository, backupLocation, BackupFilePaths.getBackupPropsName(backupId));
+        if (response.collection == null) {
+          response.collection = properties.getCollection();
+        }
+
+        // TODO Make BackupProperties itself Jackson-aware to avoid the additional conversion here?
+        Map<String, Object> details = properties.getDetails();
+        details.put(BACKUP_ID, backupId.id);
+        response.backups.add(objectMapper.convertValue(details, CollectionBackupDetails.class));
+      }
+    }
+    return response;
+  }
+
+  public static class ListCollectionBackupsResponse extends SolrJerseyResponse {
+    @JsonProperty public String collection;
+    @JsonProperty public List<CollectionBackupDetails> backups;
+  }
+
+  // TODO Merge with CreateCollectionBackupAPI.CollectionBackupDetails, which seems very
+  // conceptually similar...
+  public static class CollectionBackupDetails implements JacksonReflectMapWriter {
+    @JsonProperty public Integer backupId;
+    @JsonProperty public String indexVersion;
+    @JsonProperty public String startTime;
+    @JsonProperty public String endTime;
+    @JsonProperty public Integer indexFileCount;
+    @JsonProperty public Double indexSizeMB;
+
+    @JsonProperty public Map<String, String> shardBackupIds;
+
+    @JsonProperty(COLL_CONF)
+    public String configsetName;
+
+    @JsonProperty public String collectionAlias;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java
index f61cc312882..acd9ccf3a03 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java
@@ -38,8 +38,6 @@ import static org.apache.solr.handler.admin.CollectionsHandler.DEFAULT_COLLECTIO
 import static org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
-import java.io.IOException;
-import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
@@ -55,7 +53,6 @@ import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.jersey.JacksonReflectMapWriter;
 import org.apache.solr.jersey.PermissionName;
@@ -70,7 +67,7 @@ import org.apache.solr.response.SolrQueryResponse;
  * <p>This API is analogous to the v1 /admin/collections?action=RESTORE command.
  */
 @Path("/backups/{backupName}/restore")
-public class RestoreCollectionAPI extends AdminAPIBase {
+public class RestoreCollectionAPI extends BackupAPIBase {
 
   private static final Set<String> CREATE_PARAM_ALLOWLIST =
       Set.of(
@@ -119,23 +116,8 @@ public class RestoreCollectionAPI extends AdminAPIBase {
           "Collection '" + collectionName + "' is an existing alias, no action taken.");
     }
 
-    final BackupRepository repository = coreContainer.newBackupRepository(requestBody.repository);
     requestBody.location =
-        CreateCollectionBackupAPI.getLocation(coreContainer, repository, requestBody.location);
-
-    // Check if the specified location is valid for this repository.
-    final URI uri = repository.createDirectoryURI(requestBody.location);
-    try {
-      if (!repository.exists(uri)) {
-        throw new SolrException(
-            SolrException.ErrorCode.SERVER_ERROR, "specified location " + uri + " does not exist.");
-      }
-    } catch (IOException ex) {
-      throw new SolrException(
-          SolrException.ErrorCode.SERVER_ERROR,
-          "Failed to check the existence of " + uri + ". Is it valid?",
-          ex);
-    }
+        getAndValidateBackupLocation(requestBody.repository, requestBody.location);
 
     final var createRequestBody = requestBody.createCollectionParams;
     if (createRequestBody != null) {
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/V2CollectionBackupsAPIMappingTest.java b/solr/core/src/test/org/apache/solr/handler/admin/V2CollectionBackupsAPIMappingTest.java
deleted file mode 100644
index a4ff7d2ebe2..00000000000
--- a/solr/core/src/test/org/apache/solr/handler/admin/V2CollectionBackupsAPIMappingTest.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.handler.admin;
-
-import static org.apache.solr.common.params.CommonParams.ACTION;
-import static org.apache.solr.common.params.CommonParams.NAME;
-
-import org.apache.solr.common.params.CollectionParams;
-import org.apache.solr.common.params.CommonAdminParams;
-import org.apache.solr.common.params.CoreAdminParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.handler.CollectionBackupsAPI;
-import org.junit.Test;
-
-public class V2CollectionBackupsAPIMappingTest extends V2ApiMappingTest<CollectionsHandler> {
-  @Override
-  public void populateApiBag() {
-    final CollectionBackupsAPI collBackupsAPI = new CollectionBackupsAPI(getRequestHandler());
-    apiBag.registerObject(collBackupsAPI);
-  }
-
-  @Override
-  public CollectionsHandler createUnderlyingRequestHandler() {
-    return createMock(CollectionsHandler.class);
-  }
-
-  @Override
-  public boolean isCoreSpecific() {
-    return false;
-  }
-
-  @Test
-  public void testDeleteBackupsAllParams() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/backups",
-            "POST",
-            "{'delete-backups': {"
-                + "'name': 'backupName', "
-                + "'collection': 'collectionName', "
-                + "'location': '/some/location/uri', "
-                + "'repository': 'someRepository', "
-                + "'backupId': 123, "
-                + "'maxNumBackupPoints': 456, "
-                + "'purgeUnused': true, "
-                + "'async': 'requestTrackingId'"
-                + "}}");
-
-    assertEquals(CollectionParams.CollectionAction.DELETEBACKUP.lowerName, v1Params.get(ACTION));
-    assertEquals("backupName", v1Params.get(NAME));
-    assertEquals("/some/location/uri", v1Params.get(CoreAdminParams.BACKUP_LOCATION));
-    assertEquals("someRepository", v1Params.get(CoreAdminParams.BACKUP_REPOSITORY));
-    assertEquals(123, v1Params.getPrimitiveInt(CoreAdminParams.BACKUP_ID));
-    assertEquals(456, v1Params.getPrimitiveInt(CoreAdminParams.MAX_NUM_BACKUP_POINTS));
-    assertTrue(v1Params.getPrimitiveBool(CoreAdminParams.BACKUP_PURGE_UNUSED));
-    assertEquals("requestTrackingId", v1Params.get(CommonAdminParams.ASYNC));
-  }
-
-  @Test
-  public void testListBackupsAllParams() throws Exception {
-    final SolrParams v1Params =
-        captureConvertedV1Params(
-            "/collections/backups",
-            "POST",
-            "{'list-backups': {"
-                + "'name': 'backupName', "
-                + "'location': '/some/location/uri', "
-                + "'repository': 'someRepository' "
-                + "}}");
-
-    assertEquals(CollectionParams.CollectionAction.LISTBACKUP.lowerName, v1Params.get(ACTION));
-    assertEquals("backupName", v1Params.get(NAME));
-    assertEquals("/some/location/uri", v1Params.get(CoreAdminParams.BACKUP_LOCATION));
-    assertEquals("someRepository", v1Params.get(CoreAdminParams.BACKUP_REPOSITORY));
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/DeleteCollectionBackupAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/DeleteCollectionBackupAPITest.java
new file mode 100644
index 00000000000..02b90ba8384
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/DeleteCollectionBackupAPITest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.handler.admin.api;
+
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static org.apache.solr.common.params.CommonParams.NAME;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_ID;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_LOCATION;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_PURGE_UNUSED;
+import static org.apache.solr.common.params.CoreAdminParams.BACKUP_REPOSITORY;
+import static org.apache.solr.common.params.CoreAdminParams.MAX_NUM_BACKUP_POINTS;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.junit.Test;
+
+/** Unit tests for {@link DeleteCollectionBackupAPI} */
+public class DeleteCollectionBackupAPITest extends SolrTestCaseJ4 {
+  @Test
+  public void testReportsErrorIfBackupNameMissing() {
+    // Single delete
+    {
+      final SolrException thrown =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                final var api = new DeleteCollectionBackupAPI(null, null, null);
+                api.deleteSingleBackupById(
+                    null, "someBackupId", "someLocation", "someRepository", "someAsyncId");
+              });
+
+      assertEquals(400, thrown.code());
+      assertEquals("Missing required parameter: name", thrown.getMessage());
+    }
+
+    // Multi delete
+    {
+      final SolrException thrown =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                final var api = new DeleteCollectionBackupAPI(null, null, null);
+                api.deleteMultipleBackupsByRecency(
+                    null, 123, "someLocation", "someRepository", "someAsyncId");
+              });
+
+      assertEquals(400, thrown.code());
+      assertEquals("Missing required parameter: name", thrown.getMessage());
+    }
+
+    // Garbage collect unused files
+    {
+      final var requestBody = new DeleteCollectionBackupAPI.PurgeUnusedFilesRequestBody();
+      requestBody.location = "someLocation";
+      requestBody.repositoryName = "someRepository";
+      requestBody.asyncId = "someAsyncId";
+      final SolrException thrown =
+          expectThrows(
+              SolrException.class,
+              () -> {
+                final var api = new DeleteCollectionBackupAPI(null, null, null);
+                api.garbageCollectUnusedBackupFiles(null, requestBody);
+              });
+
+      assertEquals(400, thrown.code());
+      assertEquals("Missing required parameter: name", thrown.getMessage());
+    }
+  }
+
+  @Test
+  public void testDeletionByIdReportsErrorIfIdMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new DeleteCollectionBackupAPI(null, null, null);
+              api.deleteSingleBackupById(
+                  "someBackupName", null, "someLocation", "someRepository", "someAsyncId");
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: backupId", thrown.getMessage());
+  }
+
+  @Test
+  public void testMultiVersionDeletionReportsErrorIfRetainParamMissing() {
+    final SolrException thrown =
+        expectThrows(
+            SolrException.class,
+            () -> {
+              final var api = new DeleteCollectionBackupAPI(null, null, null);
+              api.deleteMultipleBackupsByRecency(
+                  "someBackupName", null, "someLocation", "someRepository", "someAsyncId");
+            });
+
+    assertEquals(400, thrown.code());
+    assertEquals("Missing required parameter: retainLatest", thrown.getMessage());
+  }
+
+  // The message created in this test isn't valid in practice, since it contains mutually-exclusive
+  // parameters, but that doesn't matter for the purposes of this test.
+  @Test
+  public void testCreateRemoteMessageAllParams() {
+    final var remoteMessage =
+        DeleteCollectionBackupAPI.createRemoteMessage(
+                "someBackupName",
+                "someBackupId",
+                123,
+                true,
+                "someLocation",
+                "someRepository",
+                "someAsyncId")
+            .getProperties();
+
+    assertEquals(8, remoteMessage.size());
+    assertEquals("deletebackup", remoteMessage.get(QUEUE_OPERATION));
+    assertEquals("someBackupName", remoteMessage.get(NAME));
+    assertEquals("someBackupId", remoteMessage.get(BACKUP_ID));
+    assertEquals(Integer.valueOf(123), remoteMessage.get(MAX_NUM_BACKUP_POINTS));
+    assertEquals(Boolean.TRUE, remoteMessage.get(BACKUP_PURGE_UNUSED));
+    assertEquals("someLocation", remoteMessage.get(BACKUP_LOCATION));
+    assertEquals("someRepository", remoteMessage.get(BACKUP_REPOSITORY));
+    assertEquals("someAsyncId", remoteMessage.get(ASYNC));
+  }
+
+  @Test
+  public void testCreateRemoteMessageOnlyRequiredParams() {
+    final var remoteMessage =
+        DeleteCollectionBackupAPI.createRemoteMessage(
+                "someBackupName", "someBackupId", null, null, null, null, null)
+            .getProperties();
+
+    assertEquals(3, remoteMessage.size());
+    assertEquals("deletebackup", remoteMessage.get(QUEUE_OPERATION));
+    assertEquals("someBackupName", remoteMessage.get(NAME));
+    assertEquals("someBackupId", remoteMessage.get(BACKUP_ID));
+  }
+}
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc
index a8e5592b964..2c8de19cbc4 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/collection-management.adoc
@@ -1820,14 +1820,7 @@ http://localhost:8983/solr/admin/collections?action=LISTBACKUP&name=myBackupName
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/v2/collections/backups -H 'Content-Type: application/json' -d '
-  {
-    "list-backups" : {
-      "name": "myBackupName",
-      "location": "/path/to/my/shared/drive"
-    }
-  }
-'
+curl -X GET "http://localhost:8983/api/backups/backupName/versions?location=/path/to/my/shared/drive"
 ----
 ====
 --
@@ -2005,21 +1998,23 @@ The DELETEBACKUP API does not support the deprecated format and attempts to use
 ====
 
 Solr allows storing multiple backups for the same collection at any given logical "location".
-These backup points are each given an identifier (`backupId`) which can be used to delete them specifically with this API.
+These backup points are each given an identifier (`backupId`) which can be used to delete them individually with this API.
 Alternatively Solr can be told to keep the last `maxNumBackupPoints` backups, deleting everything else at the given location.
 Deleting backup points in these ways can orphan index files that are no longer referenced by any backup points.
 These orphaned files can be detected and deleted using the `purgeUnused` option.
-See the parameter descriptions below for more information.
 
-=== DELETEBACKUP Example
+Alternately, Solr's v2 API offers separate API endpoints for each of these backup-deletion mechanisms.
+See the examples and parameter descriptions below for more information.
 
-*Input*
+=== DELETEBACKUP Examples
+
+==== Deleting by Backup ID
 
-The following API command deletes the first backup (`backupId=0`) at the specified repository location.
+*Input*
 
 [.dynamic-tabs]
 --
-[example.tab-pane#v1deletebackup]
+[example.tab-pane#v1deletebackupsingle]
 ====
 [.tab-label]*V1 API*
 
@@ -2029,21 +2024,13 @@ http://localhost:8983/solr/admin/collections?action=DELETEBACKUP&name=myBackupNa
 ----
 ====
 
-[example.tab-pane#v2deletebackup]
+[example.tab-pane#v2deletebackupsingle]
 ====
 [.tab-label]*V2 API*
 
 [source,bash]
 ----
-curl -X POST http://localhost:8983/v2/collections/backups -H 'Content-Type: application/json' -d '
-  {
-    "delete-backups" : {
-      "name": "myBackupName",
-      "location": "/path/to/my/shared/drive",
-      "backupId": 0
-    }
-  }
-'
+curl -X DELETE "http://localhost:8983/api/backups/myBackupName/versions/0?location=/path/to/my/shared/drive"
 ----
 ====
 --
@@ -2064,6 +2051,62 @@ curl -X POST http://localhost:8983/v2/collections/backups -H 'Content-Type: appl
   "collection":"books"}
 ----
 
+==== Deleting all but N recent backups
+
+[.dynamic-tabs]
+--
+[example.tab-pane#v1deletebackupmulti]
+====
+[.tab-label]*V1 API*
+
+[source,bash]
+----
+http://localhost:8983/solr/admin/collections?action=DELETEBACKUP&name=myBackupName&location=/path/to/my/shared/drive&maxNumBackupsPoints=2
+----
+====
+
+[example.tab-pane#v2deletebackupmulti]
+====
+[.tab-label]*V2 API*
+
+[source,bash]
+----
+
+curl -X DELETE "http://localhost:8983/api/backups/myBackupName/versions?retainLatest=2&location=/path/to/my/shared/drive"
+----
+====
+--
+
+==== "Garbage collecting" unused files
+
+[.dynamic-tabs]
+--
+[example.tab-pane#v1deletebackupgc]
+====
+[.tab-label]*V1 API*
+
+[source,bash]
+----
+http://localhost:8983/solr/admin/collections?action=DELETEBACKUP&name=myBackupName&location=/path/to/my/shared/drive&purgeUnused=true
+----
+====
+
+[example.tab-pane#v2deletebackupgc]
+====
+[.tab-label]*V2 API*
+
+[source,bash]
+----
+
+curl -X PUT -H "Content-type: application/json" "http://localhost:8983/api/backups/myBackupName/purgeUnused" '
+  {
+    "location": "/path/to/my/shared/drive"
+  }
+'
+----
+====
+--
+
 === DELETEBACKUP Parameters
 
 `name`::
@@ -2074,6 +2117,7 @@ s|Required |Default: none
 |===
 +
 The backup name to delete backup files from.
+Provided as a query parameter in v1 requests, and as a path parameter in v2 requests.
 
 `location`::
 +
@@ -2108,15 +2152,17 @@ If no repository is specified then the local filesystem repository will be used
 |===
 +
 Explicitly specify a single backup-ID to delete.
-Only one of `backupId`, `maxNumBackupPoints`, and `purgeUnused` may be specified per DELETEBACKUP request.
+Only one of `backupId`, `maxNumBackupPoints`, and `purgeUnused` may be specified per v1 DELETEBACKUP request.
+Provided as a query parameter in v1 requests, and as a path parameter in v2 requests.
 
-`maxNumBackupPoints`::
+`maxNumBackupPoints` (v1), `retainLatest` (v2)::
 +
 [%autowidth,frame=none]
 |===
 |Optional |Default: none
 |===
 +
+TODO
 Specify how many backups should be retained, deleting all others.
 Only one of `backupId`, `maxNumBackupPoints`, and `purgeUnused` may be specified per DELETEBACKUP request.
 
@@ -2130,7 +2176,7 @@ Only one of `backupId`, `maxNumBackupPoints`, and `purgeUnused` may be specified
 Solr's incremental backup support can orphan files if the backups referencing them are deleted.
 The `purgeUnused` flag parameter triggers a scan to detect these orphaned files and delete them.
 Administrators doing repeated backups at the same location should plan on using this parameter sporadically to reclaim disk space.
-Only one of `backupId`, `maxNumBackupPoints`, and `purgeUnused` may be specified per DELETEBACKUP request.
+Only one of `backupId`, `maxNumBackupPoints`, and `purgeUnused` may be specified per v1 DELETEBACKUP request.
 
 `async`::
 +
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 33df04ca7b2..0b2357dff0c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -40,8 +40,6 @@ import org.apache.solr.client.solrj.RoutedAliasTypes;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.request.beans.DeleteBackupPayload;
-import org.apache.solr.client.solrj.request.beans.ListBackupPayload;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
@@ -3068,13 +3066,18 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
    * "incremental=false" flag).
    */
   public static class DeleteBackup extends CollectionAdminRequest<CollectionAdminResponse> {
-    private final DeleteBackupPayload deleteBackupPayload;
+
+    private final String name;
+    private String repository;
+    private String location;
+    private Integer backupId;
+    private Integer maxNumBackupPoints;
+    private Boolean purgeUnused;
 
     private DeleteBackup(String backupName) {
       super(CollectionAction.DELETEBACKUP);
 
-      deleteBackupPayload = new DeleteBackupPayload();
-      deleteBackupPayload.name = backupName;
+      this.name = backupName;
     }
 
     /**
@@ -3082,7 +3085,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      *     information. Defaults to 'LocalFileSystemRepository' if not specified.
      */
     public DeleteBackup setRepositoryName(String backupRepository) {
-      deleteBackupPayload.repository = backupRepository;
+      this.repository = backupRepository;
       return this;
     }
 
@@ -3094,7 +3097,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      *     thrown.
      */
     public DeleteBackup setLocation(String backupLocation) {
-      deleteBackupPayload.location = backupLocation;
+      location = backupLocation;
       return this;
     }
 
@@ -3105,7 +3108,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      * @see #deleteBackupById(String, int)
      */
     protected DeleteBackup setBackupId(int backupId) {
-      deleteBackupPayload.backupId = backupId;
+      this.backupId = backupId;
       return this;
     }
 
@@ -3116,7 +3119,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      * @see #deleteBackupByRecency(String, int)
      */
     protected DeleteBackup setMaxNumBackupPoints(int backupPointsToRetain) {
-      deleteBackupPayload.maxNumBackupPoints = backupPointsToRetain;
+      this.maxNumBackupPoints = backupPointsToRetain;
       return this;
     }
 
@@ -3129,20 +3132,19 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      * @see #deleteBackupPurgeUnusedFiles(String)
      */
     protected DeleteBackup setPurgeUnused() {
-      deleteBackupPayload.purgeUnused = true;
+      this.purgeUnused = true;
       return this;
     }
 
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
-      params.set(CoreAdminParams.NAME, deleteBackupPayload.name);
-      params.setNonNull(CoreAdminParams.BACKUP_LOCATION, deleteBackupPayload.location);
-      params.setNonNull(BACKUP_REPOSITORY, deleteBackupPayload.repository);
-      params.setNonNull(CoreAdminParams.BACKUP_ID, deleteBackupPayload.backupId);
-      params.setNonNull(
-          CoreAdminParams.MAX_NUM_BACKUP_POINTS, deleteBackupPayload.maxNumBackupPoints);
-      params.setNonNull(CoreAdminParams.BACKUP_PURGE_UNUSED, deleteBackupPayload.purgeUnused);
+      params.set(CoreAdminParams.NAME, name);
+      params.setNonNull(CoreAdminParams.BACKUP_LOCATION, location);
+      params.setNonNull(BACKUP_REPOSITORY, repository);
+      params.setNonNull(CoreAdminParams.BACKUP_ID, backupId);
+      params.setNonNull(CoreAdminParams.MAX_NUM_BACKUP_POINTS, maxNumBackupPoints);
+      params.setNonNull(CoreAdminParams.BACKUP_PURGE_UNUSED, purgeUnused);
       return params;
     }
 
@@ -3171,13 +3173,14 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
    * "incremental=false" flag).
    */
   public static class ListBackup extends CollectionAdminRequest<CollectionAdminResponse> {
-    private final ListBackupPayload listPayload;
+    private final String backupName;
+    private String location;
+    private String repositoryName;
 
     private ListBackup(String backupName) {
       super(CollectionAction.LISTBACKUP);
 
-      this.listPayload = new ListBackupPayload();
-      this.listPayload.name = backupName;
+      this.backupName = backupName;
     }
 
     /**
@@ -3185,7 +3188,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      *     information. Defaults to 'LocalFileSystemRepository' if not specified.
      */
     public ListBackup setBackupRepository(String backupRepository) {
-      listPayload.repository = backupRepository;
+      this.repositoryName = backupRepository;
       return this;
     }
 
@@ -3197,16 +3200,16 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
      *     thrown.
      */
     public ListBackup setBackupLocation(String backupLocation) {
-      listPayload.location = backupLocation;
+      this.location = backupLocation;
       return this;
     }
 
     @Override
     public SolrParams getParams() {
       ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
-      params.set(CoreAdminParams.NAME, listPayload.name);
-      params.setNonNull(CoreAdminParams.BACKUP_LOCATION, listPayload.location);
-      params.setNonNull(BACKUP_REPOSITORY, listPayload.repository);
+      params.set(CoreAdminParams.NAME, backupName);
+      params.setNonNull(CoreAdminParams.BACKUP_LOCATION, this.location);
+      params.setNonNull(BACKUP_REPOSITORY, this.repositoryName);
 
       return params;
     }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteBackupPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteBackupPayload.java
deleted file mode 100644
index 98e54397f48..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/DeleteBackupPayload.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.request.beans;
-
-import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
-
-public class DeleteBackupPayload implements ReflectMapWriter {
-
-  @JsonProperty(required = true)
-  public String name;
-
-  @JsonProperty public String location;
-
-  @JsonProperty public String repository;
-
-  @JsonProperty public Integer backupId;
-
-  @JsonProperty public Integer maxNumBackupPoints;
-
-  @JsonProperty public Boolean purgeUnused;
-
-  @JsonProperty public String async;
-}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ListBackupPayload.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ListBackupPayload.java
deleted file mode 100644
index 96c0fc7106c..00000000000
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/ListBackupPayload.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.client.solrj.request.beans;
-
-import org.apache.solr.common.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
-
-public class ListBackupPayload implements ReflectMapWriter {
-
-  @JsonProperty public String name;
-
-  @JsonProperty public String location;
-
-  @JsonProperty public String repository;
-}
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractIncrementalBackupTest.java b/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractIncrementalBackupTest.java
index d43fff4f38f..8e4fb401101 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractIncrementalBackupTest.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractIncrementalBackupTest.java
@@ -57,7 +57,6 @@ import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.TrackingBackupRepository;
@@ -227,6 +226,7 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
     assertEquals(firstBatchNumDocs, getNumDocsInCollection(backupCollectionName));
   }
 
+  @SuppressWarnings("unchecked")
   @Test
   @Nightly
   public void testBackupIncremental() throws Exception {
@@ -318,7 +318,8 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
               .setRepositoryName(BACKUP_REPO_NAME)
               .setLocation(backupLocation)
               .process(cluster.getSolrClient());
-      assertEquals(2, ((NamedList) resp.getResponse().get("deleted")).get("numIndexFiles"));
+      assertEquals(
+          2, ((Map<String, Object>) resp.getResponse().get("deleted")).get("numIndexFiles"));
 
       new UpdateRequest().deleteByQuery("*:*").commit(cluster.getSolrClient(), getCollectionName());
       indexDocs(getCollectionName(), false);