CASSANDRASC-40 Fix search in list snapshot endpoint

This commit fixes test setup in SnapshotUtils. Because of the incorrect test setup
the execution is providing incorrect results. For example, assume the following path

/cassandra-test/data/ks/tbl/snapshots/test-snapshot

The test was configuring data directories as ["/cassandra-test/data"], but in a real
execution data directories is provided as ["/cassandra-test"]. This is causing the
endpoint to return incorrect values in the JSON payload.

Additionally, the response was providing the port for Cassandra and not the Sidecar
port.
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesResponse.java b/common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesResponse.java
index 5ccd609..bc57050 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesResponse.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesResponse.java
@@ -23,11 +23,15 @@
 import java.util.List;
 import java.util.Objects;
 
+import com.google.common.annotations.Beta;
+
 import com.fasterxml.jackson.annotation.JsonProperty;
 
 /**
- * A class representing a response for the {@link ListSnapshotFilesRequest}
+ * A class representing a response for the {@link ListSnapshotFilesRequest}.
+ * This class is expected to evolve and has been mark with the {@link Beta} annotation.
  */
+@Beta
 public class ListSnapshotFilesResponse
 {
     private final List<FileInfo> snapshotFilesInfo;
diff --git a/gradle.properties b/gradle.properties
index 7a32a30..fa34b46 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -1,5 +1,5 @@
 version=1.0-SNAPSHOT
 junitVersion=5.4.2
 kubernetesClientVersion=9.0.0
-cassandra40Version=4.0.4
+cassandra40Version=4.0.5
 vertxVersion=4.2.1
diff --git a/spotbugs-exclude.xml b/spotbugs-exclude.xml
index 12c771b..ee76f3c 100644
--- a/spotbugs-exclude.xml
+++ b/spotbugs-exclude.xml
@@ -17,4 +17,10 @@
         <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE" />
     </Match>
 
+    <!-- Ignore DMI_HARDCODED_ABSOLUTE_FILENAME for testing SnapshotDirectory.of with strings that are paths -->
+    <Match>
+        <Class name="org.apache.cassandra.sidecar.snapshots.SnapshotDirectoryTest" />
+        <Bug pattern="DMI_HARDCODED_ABSOLUTE_FILENAME" />
+    </Match>
+
 </FindBugsFilter>
\ No newline at end of file
diff --git a/src/main/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandler.java b/src/main/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandler.java
index 42166c9..5fe8502 100644
--- a/src/main/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandler.java
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandler.java
@@ -20,25 +20,22 @@
 
 import java.io.FileNotFoundException;
 import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.List;
 
-import org.apache.commons.lang3.tuple.Pair;
+import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.inject.Inject;
 import io.netty.handler.codec.http.HttpResponseStatus;
-import io.vertx.core.file.FileProps;
 import io.vertx.core.http.HttpServerRequest;
 import io.vertx.core.net.SocketAddress;
 import io.vertx.ext.web.RoutingContext;
 import io.vertx.ext.web.handler.HttpException;
-import org.apache.cassandra.sidecar.cluster.InstancesConfig;
-import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.Configuration;
 import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesRequest;
 import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesResponse;
+import org.apache.cassandra.sidecar.snapshots.SnapshotDirectory;
 import org.apache.cassandra.sidecar.snapshots.SnapshotPathBuilder;
 
 /**
@@ -47,37 +44,26 @@
  * For example:
  *
  * <p>
- * /api/v1/snapshots/testSnapshot                                    lists all SSTable component files for all the
- * "testSnapshot" snapshots
- * <p>
- * /api/v1/snapshots/testSnapshot?includeSecondaryIndexFiles=true    lists all SSTable component files including
- * secondary index files for all the "testSnapshot"
- * snapshots
- * <p>
- * /api/v1/keyspace/ks/table/tbl/snapshots/testSnapshot              lists all SSTable component files for the
- * "testSnapshot" snapshot for the "ks" keyspace
- * and the "tbl" table
+ * /api/v1/keyspace/ks/table/tbl/snapshots/testSnapshot
+ * lists all SSTable component files for the "testSnapshot" snapshot for the "ks" keyspace and the "tbl" table
  * <p>
  * /api/v1/keyspace/ks/table/tbl/snapshots/testSnapshot?includeSecondaryIndexFiles=true
- * lists all SSTable component files including
- * secondary index files for the "testSnapshot"
- * snapshot for the "ks" keyspace and the "tbl"
- * table
+ * lists all SSTable component files including secondary index files for the "testSnapshot" snapshot for the "ks"
+ * keyspace and the "tbl" table
  */
 public class ListSnapshotFilesHandler extends AbstractHandler
 {
     private static final Logger logger = LoggerFactory.getLogger(ListSnapshotFilesHandler.class);
     private static final String INCLUDE_SECONDARY_INDEX_FILES = "includeSecondaryIndexFiles";
-    private static final int DATA_DIR_INDEX = 0;
-    private static final int TABLE_NAME_SUBPATH_INDEX = 1;
-    private static final int FILE_NAME_SUBPATH_INDEX = 4;
     private final SnapshotPathBuilder builder;
+    private final Configuration configuration;
 
     @Inject
-    public ListSnapshotFilesHandler(SnapshotPathBuilder builder, InstancesConfig instancesConfig)
+    public ListSnapshotFilesHandler(SnapshotPathBuilder builder, Configuration configuration)
     {
-        super(instancesConfig);
+        super(configuration.getInstancesConfig());
         this.builder = builder;
+        this.configuration = configuration;
     }
 
     @Override
@@ -93,70 +79,87 @@
         boolean secondaryIndexFiles = requestParams.includeSecondaryIndexFiles();
 
         builder.build(host, requestParams)
-               .compose(directory -> builder.listSnapshotDirectory(directory, secondaryIndexFiles))
-               .onSuccess(fileList ->
-                          {
-                              if (fileList.isEmpty())
-                              {
-                                  String payload = "Snapshot '" + requestParams.getSnapshotName() + "' not found";
-                                  context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(), payload));
-                              }
-                              else
-                              {
-                                  logger.debug("ListSnapshotFilesHandler handled {} for {}. Instance: {}",
-                                               requestParams, remoteAddress, host);
-                                  context.json(buildResponse(host, requestParams, fileList));
-                              }
-                          })
-               .onFailure(cause ->
-                          {
-                              logger.error("ListSnapshotFilesHandler failed for request: {} from: {}. Instance: {}",
-                                           requestParams, remoteAddress, host);
-                              if (cause instanceof FileNotFoundException ||
-                                  cause instanceof NoSuchFileException)
-                              {
-                                  context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(),
-                                                                 cause.getMessage()));
-                              }
-                              else
-                              {
-                                  context.fail(new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
-                                                                 "Invalid request for " + requestParams));
-                              }
-                          });
+               .onSuccess(snapshotDirectory ->
+                          builder.listSnapshotDirectory(snapshotDirectory, secondaryIndexFiles)
+                                 .onSuccess(fileList ->
+                                 {
+                                     if (fileList.isEmpty())
+                                     {
+                                         String payload = "Snapshot '" + requestParams.getSnapshotName() +
+                                                          "' not found";
+                                         context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(), payload));
+                                     }
+                                     else
+                                     {
+                                         logger.debug("ListSnapshotFilesHandler handled {} for {}. Instance: {}",
+                                                      requestParams, remoteAddress, host);
+                                         context.json(buildResponse(host, snapshotDirectory, fileList));
+                                     }
+                                 })
+                                 .onFailure(cause -> processFailure(cause, context, requestParams, remoteAddress, host))
+               )
+               .onFailure(cause -> processFailure(cause, context, requestParams, remoteAddress, host));
+    }
+
+    private void processFailure(Throwable cause, RoutingContext context, ListSnapshotFilesRequest requestParams,
+                                SocketAddress remoteAddress, String host)
+    {
+        logger.error("ListSnapshotFilesHandler failed for request: {} from: {}. Instance: {}",
+                     requestParams, remoteAddress, host);
+        if (cause instanceof FileNotFoundException ||
+            cause instanceof NoSuchFileException)
+        {
+            context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(),
+                                           cause.getMessage()));
+        }
+        else
+        {
+            context.fail(new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
+                                           "Invalid request for " + requestParams));
+        }
     }
 
     private ListSnapshotFilesResponse buildResponse(String host,
-                                                    ListSnapshotFilesRequest request,
-                                                    List<Pair<String, FileProps>> fileList)
+                                                    String snapshotDirectory,
+                                                    List<SnapshotPathBuilder.SnapshotFile> fileList)
     {
-        InstanceMetadata instanceMetadata = instancesConfig.instanceFromHost(host);
-        int sidecarPort = instanceMetadata.port();
-        Path dataDirPath = Paths.get(instanceMetadata.dataDirs().get(DATA_DIR_INDEX));
         ListSnapshotFilesResponse response = new ListSnapshotFilesResponse();
-        String snapshotName = request.getSnapshotName();
+        int sidecarPort = configuration.getPort();
+        SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+        int dataDirectoryIndex = dataDirectoryIndex(host, directory.dataDirectory);
+        int offset = snapshotDirectory.length() + 1;
 
-        for (Pair<String, FileProps> file : fileList)
+        for (SnapshotPathBuilder.SnapshotFile snapshotFile : fileList)
         {
-            Path pathFromDataDir = dataDirPath.relativize(Paths.get(file.getLeft()));
-
-            String keyspace = request.getKeyspace();
-            // table name might include a dash (-) with the table UUID so we always use it as part of the response
-            String tableName = pathFromDataDir.getName(TABLE_NAME_SUBPATH_INDEX).toString();
-            String fileName = pathFromDataDir.getName(FILE_NAME_SUBPATH_INDEX).toString();
-
-            response.addSnapshotFile(new ListSnapshotFilesResponse.FileInfo(file.getRight().size(),
-                                                                            host,
-                                                                            sidecarPort,
-                                                                            DATA_DIR_INDEX,
-                                                                            snapshotName,
-                                                                            keyspace,
-                                                                            tableName,
-                                                                            fileName));
+            int fileNameIndex = snapshotFile.path.indexOf(snapshotDirectory) + offset;
+            Preconditions.checkArgument(fileNameIndex < snapshotFile.path.length(),
+                                        "Invalid snapshot file '" + snapshotFile.path + "'");
+            response.addSnapshotFile(
+            new ListSnapshotFilesResponse.FileInfo(snapshotFile.size,
+                                                   host,
+                                                   sidecarPort,
+                                                   dataDirectoryIndex,
+                                                   directory.snapshotName,
+                                                   directory.keyspace,
+                                                   directory.tableName,
+                                                   snapshotFile.path.substring(fileNameIndex)));
         }
         return response;
     }
 
+    private int dataDirectoryIndex(String host, String dataDirectory)
+    {
+        List<String> dataDirs = instancesConfig.instanceFromHost(host).dataDirs();
+        for (int index = 0; index < dataDirs.size(); index++)
+        {
+            if (dataDirectory.startsWith(dataDirs.get(index)))
+            {
+                return index;
+            }
+        }
+        return -1;
+    }
+
     private ListSnapshotFilesRequest extractParamsOrThrow(final RoutingContext context)
     {
         boolean includeSecondaryIndexFiles =
diff --git a/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotDirectory.java b/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotDirectory.java
new file mode 100644
index 0000000..215aabf
--- /dev/null
+++ b/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotDirectory.java
@@ -0,0 +1,54 @@
+package org.apache.cassandra.sidecar.snapshots;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import com.google.common.base.Preconditions;
+
+import static org.apache.cassandra.sidecar.snapshots.SnapshotPathBuilder.SNAPSHOTS_DIR_NAME;
+
+/**
+ * An object that encapsulates the parts of a snapshot directory
+ */
+public class SnapshotDirectory
+{
+    public final String dataDirectory;
+    public final String keyspace;
+    public final String tableName;
+    public final String snapshotName;
+
+    SnapshotDirectory(String dataDirectory, String keyspace, String tableName, String snapshotName)
+    {
+        this.dataDirectory = dataDirectory;
+        this.keyspace = keyspace;
+        this.tableName = tableName;
+        this.snapshotName = snapshotName;
+    }
+
+    /**
+     * Parses a snapshot directory string into a {@link SnapshotDirectory} object. The snapshot directory
+     * has the following structure {@code /&lt;data_dir&gt;/&lt;ks&gt;/&lt;table&gt;/snapshots/&lt;snapshot_name&gt;}.
+     *
+     * @param snapshotDirectory the absolute path to the snapshot directory
+     * @return the {@link SnapshotDirectory} object representing the provided {@code snapshotDirectory}
+     */
+    public static SnapshotDirectory of(String snapshotDirectory)
+    {
+        Path snapshotDirectoryPath = Paths.get(snapshotDirectory);
+        int nameCount = snapshotDirectoryPath.getNameCount();
+        Preconditions.checkArgument(nameCount >= 5, "Invalid snapshotDirectory. " +
+                                                    "Expected at least 5 parts but found " + nameCount);
+        String snapshotName = snapshotDirectoryPath.getName(nameCount - 1).toString();
+        String snapshotDirName = snapshotDirectoryPath.getName(nameCount - 2).toString();
+        String tableName = snapshotDirectoryPath.getName(nameCount - 3).toString();
+        String keyspace = snapshotDirectoryPath.getName(nameCount - 4).toString();
+        String dataDirectory = File.separator + snapshotDirectoryPath.subpath(0, nameCount - 4);
+
+        Preconditions.checkArgument(SNAPSHOTS_DIR_NAME.equalsIgnoreCase(snapshotDirName),
+                                    "Invalid snapshotDirectory. The expected directory structure is " +
+                                    "'/<data_dir>/<ks>/<table>/snapshots/<snapshot_name>'");
+
+        return new SnapshotDirectory(dataDirectory, keyspace, tableName, snapshotName);
+    }
+}
diff --git a/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotPathBuilder.java b/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotPathBuilder.java
index 54736ca..528a1a3 100644
--- a/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotPathBuilder.java
+++ b/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotPathBuilder.java
@@ -60,7 +60,7 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(SnapshotPathBuilder.class);
     private static final String DATA_SUB_DIR = "/data";
-    public static final int SNAPSHOTS_MAX_DEPTH = 4;
+    public static final int SNAPSHOTS_MAX_DEPTH = 5;
     public static final String SNAPSHOTS_DIR_NAME = "snapshots";
     protected final Vertx vertx;
     protected final FileSystem fs;
@@ -129,10 +129,10 @@
      * @param includeSecondaryIndexFiles whether to include secondary index files
      * @return a future with a list of files inside the snapshot directory
      */
-    public Future<List<Pair<String, FileProps>>> listSnapshotDirectory(String snapshotDirectory,
-                                                                       boolean includeSecondaryIndexFiles)
+    public Future<List<SnapshotFile>> listSnapshotDirectory(String snapshotDirectory,
+                                                            boolean includeSecondaryIndexFiles)
     {
-        Promise<List<Pair<String, FileProps>>> promise = Promise.promise();
+        Promise<List<SnapshotFile>> promise = Promise.promise();
 
         // List the snapshot directory
         fs.readDir(snapshotDirectory)
@@ -158,10 +158,15 @@
                              {
 
                                  // Create a pair of path/fileProps for every regular file
-                                 List<Pair<String, FileProps>> snapshotList =
+                                 List<SnapshotFile> snapshotList =
                                  IntStream.range(0, list.size())
                                           .filter(i -> ar.<FileProps>resultAt(i).isRegularFile())
-                                          .mapToObj(i -> Pair.of(list.get(i), ar.<FileProps>resultAt(i)))
+                                          .mapToObj(i ->
+                                          {
+                                              long size = ar.<FileProps>resultAt(i).size();
+                                              return new SnapshotFile(list.get(i),
+                                                                      size);
+                                          })
                                           .collect(Collectors.toList());
 
 
@@ -205,10 +210,10 @@
                                                 .onSuccess(idx ->
                                                 {
                                                     //noinspection unchecked
-                                                    List<Pair<String, FileProps>> idxPropList =
+                                                    List<SnapshotFile> idxPropList =
                                                     idx.list()
                                                        .stream()
-                                                       .flatMap(l -> ((List<Pair<String, FileProps>>) l).stream())
+                                                       .flatMap(l -> ((List<SnapshotFile>) l).stream())
                                                        .collect(Collectors.toList());
 
                                                     // aggregate the results and return the full list
@@ -249,7 +254,6 @@
 
         return vertx.executeBlocking(promise ->
         {
-
             // a filter to keep directories ending in "/snapshots/<snapshotName>"
             BiPredicate<Path, BasicFileAttributes> filter = (path, basicFileAttributes) ->
             {
@@ -512,4 +516,19 @@
                        });
         return promise.future();
     }
+
+    /**
+     * Class representing a snapshot component file
+     */
+    public static class SnapshotFile
+    {
+        public final String path;
+        public final long size;
+
+        SnapshotFile(String path, long size)
+        {
+            this.path = path;
+            this.size = size;
+        }
+    }
 }
diff --git a/src/test/java/org/apache/cassandra/sidecar/TestModule.java b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
index bd83707..5006909 100644
--- a/src/test/java/org/apache/cassandra/sidecar/TestModule.java
+++ b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
@@ -59,15 +59,15 @@
 
     @Provides
     @Singleton
-    public Configuration configuration()
+    public Configuration configuration(InstancesConfig instancesConfig)
     {
-        return abstractConfig();
+        return abstractConfig(instancesConfig);
     }
 
-    protected Configuration abstractConfig()
+    protected Configuration abstractConfig(InstancesConfig instancesConfig)
     {
         return new Configuration.Builder()
-               .setInstancesConfig(getInstancesConfig())
+               .setInstancesConfig(instancesConfig)
                .setHost("127.0.0.1")
                .setPort(6475)
                .setHealthCheckFrequency(1000)
diff --git a/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java b/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java
index a76ce13..a9def61 100644
--- a/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java
+++ b/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java
@@ -23,6 +23,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.sidecar.cluster.InstancesConfig;
+
 /**
  * Changes to the TestModule to define SSL dependencies
  */
@@ -32,7 +34,7 @@
 
 
     @Override
-    public Configuration abstractConfig()
+    public Configuration abstractConfig(InstancesConfig instancesConfig)
     {
         final String keyStorePath = TestSslModule.class.getClassLoader().getResource("certs/test.p12").getPath();
         final String keyStorePassword = "password";
@@ -50,7 +52,7 @@
         }
 
         return new Configuration.Builder()
-                           .setInstancesConfig(getInstancesConfig())
+                           .setInstancesConfig(instancesConfig)
                            .setHost("127.0.0.1")
                            .setPort(6475)
                            .setHealthCheckFrequency(1000)
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandlerTest.java
index 7c959c9..9f9765e 100644
--- a/src/test/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandlerTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandlerTest.java
@@ -20,6 +20,8 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -34,6 +36,8 @@
 import com.google.inject.AbstractModule;
 import com.google.inject.Guice;
 import com.google.inject.Injector;
+import com.google.inject.Provides;
+import com.google.inject.Singleton;
 import com.google.inject.util.Modules;
 import io.vertx.core.Vertx;
 import io.vertx.core.http.HttpServer;
@@ -103,7 +107,7 @@
         ListSnapshotFilesResponse.FileInfo fileInfoExpected =
         new ListSnapshotFilesResponse.FileInfo(11,
                                                "localhost",
-                                               9043,
+                                               6475,
                                                0,
                                                "snapshot1",
                                                "keyspace1",
@@ -112,7 +116,7 @@
         ListSnapshotFilesResponse.FileInfo fileInfoNotExpected =
         new ListSnapshotFilesResponse.FileInfo(11,
                                                "localhost",
-                                               9043,
+                                               6475,
                                                0,
                                                "snapshot1",
                                                "keyspace1",
@@ -132,6 +136,51 @@
     }
 
     @Test
+    public void testRouteSucceedsIncludeSecondaryIndexes(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        String testRoute = "/api/v1/keyspace/keyspace1/table/table1-1234" +
+                           "/snapshots/snapshot1?includeSecondaryIndexFiles=true";
+        List<ListSnapshotFilesResponse.FileInfo> fileInfoExpected = Arrays.asList(
+        new ListSnapshotFilesResponse.FileInfo(11,
+                                               "localhost",
+                                               6475,
+                                               0,
+                                               "snapshot1",
+                                               "keyspace1",
+                                               "table1-1234",
+                                               "1.db"),
+        new ListSnapshotFilesResponse.FileInfo(0,
+                                               "localhost",
+                                               6475,
+                                               0,
+                                               "snapshot1",
+                                               "keyspace1",
+                                               "table1-1234",
+                                               ".index/secondary.db")
+        );
+        ListSnapshotFilesResponse.FileInfo fileInfoNotExpected =
+        new ListSnapshotFilesResponse.FileInfo(11,
+                                               "localhost",
+                                               6475,
+                                               0,
+                                               "snapshot1",
+                                               "keyspace1",
+                                               "table1-1234",
+                                               "2.db");
+
+        client.get(config.getPort(), "localhost", testRoute)
+              .send(context.succeeding(response -> context.verify(() ->
+              {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  ListSnapshotFilesResponse resp = response.bodyAsJson(ListSnapshotFilesResponse.class);
+                  assertThat(resp.getSnapshotFilesInfo()).containsAll(fileInfoExpected);
+                  assertThat(resp.getSnapshotFilesInfo()).doesNotContain(fileInfoNotExpected);
+                  context.completeNow();
+              })));
+    }
+
+    @Test
     public void testRouteInvalidSnapshot(VertxTestContext context)
     {
         WebClient client = WebClient.create(vertx);
@@ -147,17 +196,11 @@
 
     class ListSnapshotTestModule extends AbstractModule
     {
-        @Override
-        protected void configure()
+        @Provides
+        @Singleton
+        public InstancesConfig getInstancesConfig() throws IOException
         {
-            try
-            {
-                bind(InstancesConfig.class).toInstance(mockInstancesConfig(temporaryFolder.getCanonicalPath()));
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
+            return mockInstancesConfig(temporaryFolder.getCanonicalPath());
         }
     }
 }
diff --git a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotDirectoryTest.java b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotDirectoryTest.java
new file mode 100644
index 0000000..dfc0aa1
--- /dev/null
+++ b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotDirectoryTest.java
@@ -0,0 +1,74 @@
+package org.apache.cassandra.sidecar.snapshots;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException;
+
+class SnapshotDirectoryTest
+{
+
+    @ParameterizedTest
+    @ValueSource(strings = { "not-valid", "/two-levels/not-valid", "three/levels/not-valid", "four/levels/not/valid" })
+    void failsOnInvalidLengthDirectory()
+    {
+        assertThatIllegalArgumentException()
+        .isThrownBy(() -> SnapshotDirectory.of("not-valid"))
+        .withMessageContaining("Invalid snapshotDirectory. Expected at least 5 parts but found");
+    }
+
+    @Test
+    void failsOnInvalidDirectory()
+    {
+        assertThatIllegalArgumentException()
+        .isThrownBy(() -> SnapshotDirectory.of("/cassandra/data/ks1/tbl2/sneaky/test-snapshot"))
+        .withMessage("Invalid snapshotDirectory. The expected directory structure is " +
+                     "'/<data_dir>/<ks>/<table>/snapshots/<snapshot_name>'");
+    }
+
+    @Test
+    void testValidDirectory1()
+    {
+        String snapshotDirectory = "/cassandra/data/ks1/tbl2/snapshots/test-snapshot";
+        SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+        assertThat(directory.dataDirectory).isEqualTo("/cassandra/data");
+        assertThat(directory.keyspace).isEqualTo("ks1");
+        assertThat(directory.tableName).isEqualTo("tbl2");
+        assertThat(directory.snapshotName).isEqualTo("test-snapshot");
+    }
+
+    @Test
+    void testValidDirectory2()
+    {
+        String snapshotDirectory = "/cassandra/data/ks1/tbl2/SNAPSHOTS/test-snapshot";
+        SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+        assertThat(directory.dataDirectory).isEqualTo("/cassandra/data");
+        assertThat(directory.keyspace).isEqualTo("ks1");
+        assertThat(directory.tableName).isEqualTo("tbl2");
+        assertThat(directory.snapshotName).isEqualTo("test-snapshot");
+    }
+
+    @Test
+    void testValidDirectory3()
+    {
+        String snapshotDirectory = "/datadir/inventory/shipping/snapshots/2022-07-23";
+        SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+        assertThat(directory.dataDirectory).isEqualTo("/datadir");
+        assertThat(directory.keyspace).isEqualTo("inventory");
+        assertThat(directory.tableName).isEqualTo("shipping");
+        assertThat(directory.snapshotName).isEqualTo("2022-07-23");
+    }
+
+    @Test
+    void testValidDirectory4()
+    {
+        String snapshotDirectory = "/cassandra/disk1/data/inventory/shipping/snapshots/2022-07-23/";
+        SnapshotDirectory directory = SnapshotDirectory.of(snapshotDirectory);
+        assertThat(directory.dataDirectory).isEqualTo("/cassandra/disk1/data");
+        assertThat(directory.keyspace).isEqualTo("inventory");
+        assertThat(directory.tableName).isEqualTo("shipping");
+        assertThat(directory.snapshotName).isEqualTo("2022-07-23");
+    }
+}
diff --git a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotSearchTest.java b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotSearchTest.java
index 2649c32..7d10c8d 100644
--- a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotSearchTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotSearchTest.java
@@ -26,7 +26,6 @@
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -35,7 +34,6 @@
 import io.vertx.core.CompositeFuture;
 import io.vertx.core.Future;
 import io.vertx.core.Vertx;
-import io.vertx.core.file.FileProps;
 import io.vertx.junit5.VertxExtension;
 import io.vertx.junit5.VertxTestContext;
 import org.apache.cassandra.sidecar.cluster.InstancesConfig;
@@ -145,8 +143,8 @@
         //noinspection unchecked
         List<String> snapshotFiles = ar.list()
                                        .stream()
-                                       .flatMap(l -> ((List<Pair<String, FileProps>>) l).stream())
-                                       .map(Pair::getLeft)
+                                       .flatMap(l -> ((List<SnapshotPathBuilder.SnapshotFile>) l).stream())
+                                       .map(snapshotFile -> snapshotFile.path)
                                        .sorted()
                                        .collect(Collectors.toList());
 
diff --git a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java
index f16605d..d482fbd 100644
--- a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java
+++ b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java
@@ -81,14 +81,14 @@
         InstanceMetadataImpl localhost = new InstanceMetadataImpl(1,
                                                                   "localhost",
                                                                   9043,
-                                                                  Collections.singletonList(rootPath + "/d1/data"),
+                                                                  Collections.singletonList(rootPath + "/d1"),
                                                                   null,
                                                                   versionProvider,
                                                                   1000);
         InstanceMetadataImpl localhost2 = new InstanceMetadataImpl(2,
                                                                    "localhost2",
                                                                    9043,
-                                                                   Collections.singletonList(rootPath + "/d2/data"),
+                                                                   Collections.singletonList(rootPath + "/d2"),
                                                                    null,
                                                                    versionProvider,
                                                                    1000);