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/07/05 11:56:52 UTC

[solr] 01/02: SOLR-16470 : Create V2 equivalent of V1 Replication: GET files (#1704)

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 3847696e38a870f6430e693237f178767695f994
Author: Matthew Biscocho <54...@users.noreply.github.com>
AuthorDate: Fri Jun 30 13:00:19 2023 -0400

    SOLR-16470 : Create V2 equivalent of V1 Replication: GET files (#1704)
    
    No v2 equivalent existed prior to this commit.  The new v2 API is
    `GET /api/cores/cName/replication/files?generation=<123>`.
    ---------
    
    Co-authored-by: Jason Gerlowski <ge...@apache.org>
---
 solr/CHANGES.txt                                   |   3 +
 .../java/org/apache/solr/handler/IndexFetcher.java |   9 +-
 .../apache/solr/handler/ReplicationHandler.java    | 159 ++++-----------------
 .../solr/handler/admin/api/CoreReplicationAPI.java |  65 ++++++++-
 .../solr/handler/admin/api/ReplicationAPIBase.java | 151 ++++++++++++++++++-
 .../handler/admin/api/CoreReplicationAPITest.java  |  37 ++++-
 .../pages/user-managed-index-replication.adoc      |  18 +++
 7 files changed, 299 insertions(+), 143 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a540711480e..9c823b016f2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -168,6 +168,9 @@ Improvements
   specify where to download Solr from. If the url includes "apache.org", then GPG checks will be computed, otherwise
   the GPG checks will be skipped. (Houston Putman)
 
+* SOLR-16470: `/coreName/replication?command=filelist` now has a v2 equivalent, available at
+  `GET /api/cores/coreName/replication/files?generation=123` (Matthew Biscocho via Jason Gerlowski)
+
 Optimizations
 ---------------------
 
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 34f1395ec97..18a9c60fad7 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -121,6 +121,7 @@ import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.admin.api.CoreReplicationAPI;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -1606,14 +1607,14 @@ public class IndexFetcher {
       names.add(name, null);
     }
     // get the details of the local conf files with the same alias/name
-    List<Map<String, Object>> localFilesInfo =
+    List<CoreReplicationAPI.FileMetaData> localFilesInfo =
         replicationHandler.getConfFileInfoFromCache(names, confFileInfoCache);
     // compare their size/checksum to see if
-    for (Map<String, Object> fileInfo : localFilesInfo) {
-      String name = (String) fileInfo.get(NAME);
+    for (CoreReplicationAPI.FileMetaData fileInfo : localFilesInfo) {
+      String name = fileInfo.name;
       Map<String, Object> m = nameVsFile.get(name);
       if (m == null) continue; // the file is not even present locally (so must be downloaded)
-      if (m.get(CHECKSUM).equals(fileInfo.get(CHECKSUM))) {
+      if (m.get(CHECKSUM).equals(fileInfo.checksum)) {
         nameVsFile.remove(name); // checksums are same so the file need not be downloaded
       }
     }
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 3e358fd8fb7..f49ef231f9e 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -56,13 +56,10 @@ import java.util.zip.Adler32;
 import java.util.zip.Checksum;
 import java.util.zip.DeflaterOutputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.SegmentCommitInfo;
-import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -272,7 +269,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     } else if (command.equals(CMD_GET_FILE)) {
       getFileStream(solrParams, rsp);
     } else if (command.equals(CMD_GET_FILE_LIST)) {
-      getFileList(solrParams, rsp);
+      final CoreReplicationAPI coreReplicationAPI = new CoreReplicationAPI(core, req, rsp);
+      V2ApiUtils.squashIntoSolrResponseWithoutHeader(
+          rsp,
+          coreReplicationAPI.fetchFileList(Long.parseLong(solrParams.required().get(GENERATION))));
     } else if (command.equalsIgnoreCase(CMD_BACKUP)) {
       doSnapShoot(new ModifiableSolrParams(solrParams), rsp, req);
     } else if (command.equalsIgnoreCase(CMD_RESTORE)) {
@@ -670,113 +670,6 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     rsp.add(STATUS, OK_STATUS);
   }
 
-  private void getFileList(SolrParams solrParams, SolrQueryResponse rsp) {
-    final IndexDeletionPolicyWrapper delPol = core.getDeletionPolicy();
-    final long gen = Long.parseLong(solrParams.required().get(GENERATION));
-
-    IndexCommit commit = null;
-    try {
-      if (gen == -1) {
-        commit = delPol.getAndSaveLatestCommit();
-        if (null == commit) {
-          rsp.add(CMD_GET_FILE_LIST, Collections.emptyList());
-          return;
-        }
-      } else {
-        try {
-          commit = delPol.getAndSaveCommitPoint(gen);
-        } catch (IllegalStateException ignored) {
-          /* handle this below the same way we handle a return value of null... */
-        }
-        if (null == commit) {
-          // The gen they asked for either doesn't exist or has already been deleted
-          reportErrorOnResponse(rsp, "invalid index generation", null);
-          return;
-        }
-      }
-      assert null != commit;
-
-      List<Map<String, Object>> result = new ArrayList<>();
-      Directory dir = null;
-      try {
-        dir =
-            core.getDirectoryFactory()
-                .get(
-                    core.getNewIndexDir(),
-                    DirContext.DEFAULT,
-                    core.getSolrConfig().indexConfig.lockType);
-        SegmentInfos infos = SegmentInfos.readCommit(dir, commit.getSegmentsFileName());
-        for (SegmentCommitInfo commitInfo : infos) {
-          for (String file : commitInfo.files()) {
-            Map<String, Object> fileMeta = new HashMap<>();
-            fileMeta.put(NAME, file);
-            fileMeta.put(SIZE, dir.fileLength(file));
-
-            try (final IndexInput in = dir.openInput(file, IOContext.READONCE)) {
-              try {
-                long checksum = CodecUtil.retrieveChecksum(in);
-                fileMeta.put(CHECKSUM, checksum);
-              } catch (Exception e) {
-                // TODO Should this trigger a larger error?
-                log.warn("Could not read checksum from index file: {}", file, e);
-              }
-            }
-
-            result.add(fileMeta);
-          }
-        }
-
-        // add the segments_N file
-
-        Map<String, Object> fileMeta = new HashMap<>();
-        fileMeta.put(NAME, infos.getSegmentsFileName());
-        fileMeta.put(SIZE, dir.fileLength(infos.getSegmentsFileName()));
-        if (infos.getId() != null) {
-          try (final IndexInput in =
-              dir.openInput(infos.getSegmentsFileName(), IOContext.READONCE)) {
-            try {
-              fileMeta.put(CHECKSUM, CodecUtil.retrieveChecksum(in));
-            } catch (Exception e) {
-              // TODO Should this trigger a larger error?
-              log.warn(
-                  "Could not read checksum from index file: {}", infos.getSegmentsFileName(), e);
-            }
-          }
-        }
-        result.add(fileMeta);
-      } catch (IOException e) {
-        log.error(
-            "Unable to get file names for indexCommit generation: {}", commit.getGeneration(), e);
-        reportErrorOnResponse(rsp, "unable to get file names for given index generation", e);
-        return;
-      } finally {
-        if (dir != null) {
-          try {
-            core.getDirectoryFactory().release(dir);
-          } catch (IOException e) {
-            log.error("Could not release directory after fetching file list", e);
-          }
-        }
-      }
-      rsp.add(CMD_GET_FILE_LIST, result);
-
-      if (confFileNameAlias.size() < 1 || core.getCoreContainer().isZooKeeperAware()) return;
-      log.debug("Adding config files to list: {}", includeConfFiles);
-      // if configuration files need to be included get their details
-      rsp.add(CONF_FILES, getConfFileInfoFromCache(confFileNameAlias, confFileInfoCache));
-      rsp.add(STATUS, OK_STATUS);
-
-    } finally {
-      if (null != commit) {
-        // before releasing the save on our commit point, set a short reserve duration since
-        // the main reason remote nodes will ask for the file list is because they are preparing to
-        // replicate from us...
-        delPol.setReserveDuration(commit.getGeneration(), reserveCommitDuration);
-        delPol.releaseCommitPoint(commit);
-      }
-    }
-  }
-
   public CoreReplicationAPI.IndexVersionResponse getIndexVersionResponse() throws IOException {
 
     IndexCommit commitPoint = indexCommitPoint; // make a copy so it won't change
@@ -828,9 +721,9 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
    * <p>The local conf files information is cached so that everytime it does not have to compute the
    * checksum. The cache is refreshed only if the lastModified of the file changes
    */
-  List<Map<String, Object>> getConfFileInfoFromCache(
+  public List<CoreReplicationAPI.FileMetaData> getConfFileInfoFromCache(
       NamedList<String> nameAndAlias, final Map<String, FileInfo> confFileInfoCache) {
-    List<Map<String, Object>> confFiles = new ArrayList<>();
+    List<CoreReplicationAPI.FileMetaData> confFiles = new ArrayList<>();
     synchronized (confFileInfoCache) {
       Checksum checksum = null;
       for (int i = 0; i < nameAndAlias.size(); i++) {
@@ -846,13 +739,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         } catch (IOException e) {
           // proceed with zeroes for now, will probably error on checksum anyway
         }
-        if (info == null || info.lastmodified != lastModified || info.size != size) {
+        if (info == null || info.lastmodified != lastModified || info.fileMetaData.size != size) {
           if (checksum == null) checksum = new Adler32();
           info = new FileInfo(lastModified, cf, size, getCheckSum(checksum, f));
           confFileInfoCache.put(cf, info);
         }
-        Map<String, Object> m = info.getAsMap();
-        if (nameAndAlias.getVal(i) != null) m.put(ALIAS, nameAndAlias.getVal(i));
+        CoreReplicationAPI.FileMetaData m = info.fileMetaData;
+        if (nameAndAlias.getVal(i) != null) m.alias = nameAndAlias.getVal(i);
         confFiles.add(m);
       }
     }
@@ -861,23 +754,11 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   static class FileInfo {
     long lastmodified;
-    String name;
-    long size;
-    long checksum;
+    CoreReplicationAPI.FileMetaData fileMetaData;
 
     public FileInfo(long lasmodified, String name, long size, long checksum) {
       this.lastmodified = lasmodified;
-      this.name = name;
-      this.size = size;
-      this.checksum = checksum;
-    }
-
-    Map<String, Object> getAsMap() {
-      Map<String, Object> map = new HashMap<>();
-      map.put(NAME, name);
-      map.put(SIZE, size);
-      map.put(CHECKSUM, checksum);
-      return map;
+      this.fileMetaData = new CoreReplicationAPI.FileMetaData(size, name, checksum);
     }
   }
 
@@ -939,6 +820,22 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     return "ReplicationHandler provides replication of index and configuration files from Leader to Followers";
   }
 
+  public NamedList<String> getConfFileNameAlias() {
+    return confFileNameAlias;
+  }
+
+  public Map<String, FileInfo> getConfFileInfoCache() {
+    return confFileInfoCache;
+  }
+
+  public String getIncludeConfFiles() {
+    return includeConfFiles;
+  }
+
+  public Long getReserveCommitDuration() {
+    return reserveCommitDuration;
+  }
+
   /** returns the CommitVersionInfo for the current searcher, or null on error. */
   private CommitVersionInfo getIndexVersion() {
     try {
@@ -1876,7 +1773,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
 
   private static final String FAILED = "failed";
 
-  private static final String EXCEPTION = "exception";
+  public static final String EXCEPTION = "exception";
 
   public static final String LEADER_URL = "leaderUrl";
 
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java
index 29ead1bd519..f2c4ac60cd6 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/CoreReplicationAPI.java
@@ -20,12 +20,17 @@ import static org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONT
 import static org.apache.solr.security.PermissionNameProvider.Name.CORE_READ_PERM;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.Parameter;
 import java.io.IOException;
+import java.util.List;
 import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
 import org.apache.solr.jersey.PermissionName;
 import org.apache.solr.jersey.SolrJerseyResponse;
 import org.apache.solr.request.SolrQueryRequest;
@@ -46,12 +51,23 @@ public class CoreReplicationAPI extends ReplicationAPIBase {
 
   @GET
   @Path("/indexversion")
-  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
   @PermissionName(CORE_READ_PERM)
   public IndexVersionResponse fetchIndexVersion() throws IOException {
     return doFetchIndexVersion();
   }
 
+  @GET
+  @Path("/files")
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, BINARY_CONTENT_TYPE_V2})
+  @PermissionName(CORE_READ_PERM)
+  public FileListResponse fetchFileList(
+      @Parameter(description = "The generation number of the index", required = true)
+          @QueryParam("generation")
+          long gen) {
+    return doFetchFileList(gen);
+  }
+
   /** Response for {@link CoreReplicationAPI#fetchIndexVersion()}. */
   public static class IndexVersionResponse extends SolrJerseyResponse {
 
@@ -72,4 +88,51 @@ public class CoreReplicationAPI extends ReplicationAPIBase {
       this.status = status;
     }
   }
+
+  /** Response for {@link CoreReplicationAPI#fetchFileList(long)}. */
+  public static class FileListResponse extends SolrJerseyResponse {
+    @JsonProperty("filelist")
+    public List<FileMetaData> fileList;
+
+    @JsonProperty("confFiles")
+    public List<FileMetaData> confFiles;
+
+    @JsonProperty("status")
+    public String status;
+
+    @JsonProperty("message")
+    public String message;
+
+    @JsonProperty("exception")
+    public Exception exception;
+
+    public FileListResponse() {}
+  }
+
+  /**
+   * Contained in {@link CoreReplicationAPI.FileListResponse}, this holds metadata from a file for
+   * an index
+   */
+  public static class FileMetaData implements JacksonReflectMapWriter {
+
+    @JsonProperty("size")
+    public long size;
+
+    @JsonProperty("name")
+    public String name;
+
+    @JsonProperty("checksum")
+    public long checksum;
+
+    @JsonProperty("alias")
+    public String alias;
+
+    public FileMetaData() {}
+
+    public FileMetaData(long size, String name, long checksum) {
+      this.size = size;
+      this.name = name;
+      this.checksum = checksum;
+    }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java b/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java
index e53261350ff..f438ecc2ac7 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/ReplicationAPIBase.java
@@ -16,16 +16,35 @@
  */
 package org.apache.solr.handler.admin.api;
 
+import static org.apache.solr.handler.ReplicationHandler.ERR_STATUS;
+import static org.apache.solr.handler.ReplicationHandler.OK_STATUS;
+
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 import org.apache.solr.api.JerseyResource;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** A common parent for "replication" (i.e. replication-level) APIs. */
 public abstract class ReplicationAPIBase extends JerseyResource {
 
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final SolrCore solrCore;
   protected final SolrQueryRequest solrQueryRequest;
   protected final SolrQueryResponse solrQueryResponse;
@@ -38,10 +57,140 @@ public abstract class ReplicationAPIBase extends JerseyResource {
   }
 
   protected CoreReplicationAPI.IndexVersionResponse doFetchIndexVersion() throws IOException {
+    ReplicationHandler replicationHandler =
+        (ReplicationHandler) solrCore.getRequestHandler(ReplicationHandler.PATH);
+    return replicationHandler.getIndexVersionResponse();
+  }
 
+  protected CoreReplicationAPI.FileListResponse doFetchFileList(long generation) {
     ReplicationHandler replicationHandler =
         (ReplicationHandler) solrCore.getRequestHandler(ReplicationHandler.PATH);
+    return getFileList(generation, replicationHandler);
+  }
 
-    return replicationHandler.getIndexVersionResponse();
+  protected CoreReplicationAPI.FileListResponse getFileList(
+      long generation, ReplicationHandler replicationHandler) {
+    final IndexDeletionPolicyWrapper delPol = solrCore.getDeletionPolicy();
+    final CoreReplicationAPI.FileListResponse filesResponse =
+        new CoreReplicationAPI.FileListResponse();
+
+    IndexCommit commit = null;
+    try {
+      if (generation == -1) {
+        commit = delPol.getAndSaveLatestCommit();
+        if (null == commit) {
+          filesResponse.fileList = Collections.emptyList();
+          return filesResponse;
+        }
+      } else {
+        try {
+          commit = delPol.getAndSaveCommitPoint(generation);
+        } catch (IllegalStateException ignored) {
+          /* handle this below the same way we handle a return value of null... */
+        }
+        if (null == commit) {
+          // The gen they asked for either doesn't exist or has already been deleted
+          reportErrorOnResponse(filesResponse, "invalid index generation", null);
+          return filesResponse;
+        }
+      }
+      assert null != commit;
+
+      List<CoreReplicationAPI.FileMetaData> result = new ArrayList<>();
+      Directory dir = null;
+      try {
+        dir =
+            solrCore
+                .getDirectoryFactory()
+                .get(
+                    solrCore.getNewIndexDir(),
+                    DirectoryFactory.DirContext.DEFAULT,
+                    solrCore.getSolrConfig().indexConfig.lockType);
+        SegmentInfos infos = SegmentInfos.readCommit(dir, commit.getSegmentsFileName());
+        for (SegmentCommitInfo commitInfo : infos) {
+          for (String file : commitInfo.files()) {
+            CoreReplicationAPI.FileMetaData metaData = new CoreReplicationAPI.FileMetaData();
+            metaData.name = file;
+            metaData.size = dir.fileLength(file);
+
+            try (final IndexInput in = dir.openInput(file, IOContext.READONCE)) {
+              try {
+                long checksum = CodecUtil.retrieveChecksum(in);
+                metaData.checksum = checksum;
+              } catch (Exception e) {
+                // TODO Should this trigger a larger error?
+                log.warn("Could not read checksum from index file: {}", file, e);
+              }
+            }
+            result.add(metaData);
+          }
+        }
+
+        // add the segments_N file
+        CoreReplicationAPI.FileMetaData fileMetaData = new CoreReplicationAPI.FileMetaData();
+        fileMetaData.name = infos.getSegmentsFileName();
+        fileMetaData.size = dir.fileLength(infos.getSegmentsFileName());
+        if (infos.getId() != null) {
+          try (final IndexInput in =
+              dir.openInput(infos.getSegmentsFileName(), IOContext.READONCE)) {
+            try {
+              fileMetaData.checksum = CodecUtil.retrieveChecksum(in);
+            } catch (Exception e) {
+              // TODO Should this trigger a larger error?
+              log.warn(
+                  "Could not read checksum from index file: {}", infos.getSegmentsFileName(), e);
+            }
+          }
+        }
+        result.add(fileMetaData);
+      } catch (IOException e) {
+        log.error(
+            "Unable to get file names for indexCommit generation: {}", commit.getGeneration(), e);
+        reportErrorOnResponse(
+            filesResponse, "unable to get file names for given index generation", e);
+        return filesResponse;
+      } finally {
+        if (dir != null) {
+          try {
+            solrCore.getDirectoryFactory().release(dir);
+          } catch (IOException e) {
+            log.error("Could not release directory after fetching file list", e);
+          }
+        }
+      }
+      filesResponse.fileList = new ArrayList<>(result);
+
+      if (replicationHandler.getConfFileNameAlias().size() < 1
+          || solrCore.getCoreContainer().isZooKeeperAware()) return filesResponse;
+      String includeConfFiles = replicationHandler.getIncludeConfFiles();
+      log.debug("Adding config files to list: {}", includeConfFiles);
+      // if configuration files need to be included get their details
+      filesResponse.confFiles =
+          new ArrayList<>(
+              replicationHandler.getConfFileInfoFromCache(
+                  replicationHandler.getConfFileNameAlias(),
+                  replicationHandler.getConfFileInfoCache()));
+      filesResponse.status = OK_STATUS;
+
+    } finally {
+      if (null != commit) {
+        // before releasing the save on our commit point, set a short reserve duration since
+        // the main reason remote nodes will ask for the file list is because they are preparing to
+        // replicate from us...
+        delPol.setReserveDuration(
+            commit.getGeneration(), replicationHandler.getReserveCommitDuration());
+        delPol.releaseCommitPoint(commit);
+      }
+    }
+    return filesResponse;
+  }
+
+  private void reportErrorOnResponse(
+      CoreReplicationAPI.FileListResponse fileListResponse, String message, Exception e) {
+    fileListResponse.status = ERR_STATUS;
+    fileListResponse.message = message;
+    if (e != null) {
+      fileListResponse.exception = e;
+    }
   }
 }
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java b/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java
index 98961280fa1..47faa668ba2 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/api/CoreReplicationAPITest.java
@@ -21,6 +21,9 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import io.opentracing.noop.NoopSpan;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ReplicationHandler;
@@ -36,7 +39,6 @@ public class CoreReplicationAPITest extends SolrTestCaseJ4 {
   private CoreReplicationAPI coreReplicationAPI;
   private SolrCore mockCore;
   private ReplicationHandler mockReplicationHandler;
-  private static final String coreName = "test";
   private SolrQueryRequest mockQueryRequest;
   private SolrQueryResponse queryResponse;
 
@@ -53,7 +55,7 @@ public class CoreReplicationAPITest extends SolrTestCaseJ4 {
     mockQueryRequest = mock(SolrQueryRequest.class);
     when(mockQueryRequest.getSpan()).thenReturn(NoopSpan.INSTANCE);
     queryResponse = new SolrQueryResponse();
-    coreReplicationAPI = new CoreReplicationAPI(mockCore, mockQueryRequest, queryResponse);
+    coreReplicationAPI = new CoreReplicationAPIMock(mockCore, mockQueryRequest, queryResponse);
   }
 
   @Test
@@ -62,10 +64,19 @@ public class CoreReplicationAPITest extends SolrTestCaseJ4 {
         new CoreReplicationAPI.IndexVersionResponse(123L, 123L, "testGeneration");
     when(mockReplicationHandler.getIndexVersionResponse()).thenReturn(expected);
 
-    CoreReplicationAPI.IndexVersionResponse response = coreReplicationAPI.doFetchIndexVersion();
-    assertEquals(expected.indexVersion, response.indexVersion);
-    assertEquals(expected.generation, response.generation);
-    assertEquals(expected.status, response.status);
+    CoreReplicationAPI.IndexVersionResponse actual = coreReplicationAPI.doFetchIndexVersion();
+    assertEquals(expected.indexVersion, actual.indexVersion);
+    assertEquals(expected.generation, actual.generation);
+    assertEquals(expected.status, actual.status);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testFetchFiles() throws Exception {
+    CoreReplicationAPI.FileListResponse actualResponse = coreReplicationAPI.fetchFileList(-1);
+    assertEquals(123, actualResponse.fileList.get(0).size);
+    assertEquals("test", actualResponse.fileList.get(0).name);
+    assertEquals(123456789, actualResponse.fileList.get(0).checksum);
   }
 
   private void setUpMocks() {
@@ -73,4 +84,18 @@ public class CoreReplicationAPITest extends SolrTestCaseJ4 {
     mockReplicationHandler = mock(ReplicationHandler.class);
     when(mockCore.getRequestHandler(ReplicationHandler.PATH)).thenReturn(mockReplicationHandler);
   }
+
+  class CoreReplicationAPIMock extends CoreReplicationAPI {
+    public CoreReplicationAPIMock(SolrCore solrCore, SolrQueryRequest req, SolrQueryResponse rsp) {
+      super(solrCore, req, rsp);
+    }
+
+    @Override
+    protected FileListResponse getFileList(long generation, ReplicationHandler replicationHandler) {
+      final FileListResponse filesResponse = new FileListResponse();
+      List<FileMetaData> fileMetaData = Arrays.asList(new FileMetaData(123, "test", 123456789));
+      filesResponse.fileList = new ArrayList<>(fileMetaData);
+      return filesResponse;
+    }
+  }
 }
diff --git a/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc b/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc
index 4def27755a2..0f981c557d7 100644
--- a/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc
+++ b/solr/solr-ref-guide/modules/deployment-guide/pages/user-managed-index-replication.adoc
@@ -433,9 +433,27 @@ http://_follower_host:port_/solr/_core_name_/replication?command=details
 
 `filelist`::
 Retrieve a list of Lucene files present in the specified host's index.
+
 +
+====
+[.tab-label]*V1 API*
+
 [source,bash]
+----
 http://_host:port_/solr/_core_name_/replication?command=filelist&generation=<_generation-number_>
+
+----
+====
++
+====
+[.tab-label]*V2 API*
+
+[source,bash]
+----
+http://_host:port_/api/cores/_core_name_/replication/files?generation=<_generation-number_>
+
+----
+====
 +
 You can discover the generation number of the index by running the `indexversion` command.