You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/07/05 20:54:58 UTC

[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #31: CASSANDRASC-38 Add endpoint to list snapshot files

yifan-c commented on code in PR #31:
URL: https://github.com/apache/cassandra-sidecar/pull/31#discussion_r914159174


##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesRequest.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.cassandra.sidecar.common.data;
+
+/**
+ * Holder class for the {@code org.apache.cassandra.sidecar.routes.ListSnapshotFilesHandler}

Review Comment:
   nit: `@code` --> `@link` maybe?



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesRequest.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.cassandra.sidecar.common.data;

Review Comment:
   If the data class is created for `ListSnapshotFilesHandler`, I do not see why placing it under `common` package, since it is not shared by multiple packages. It is coupled with `ListSnapshotFilesHandler`. Maybe relocated it together with the handler? 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesResponse.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.cassandra.sidecar.common.data;
+
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A class representing a response for the {@link ListSnapshotFilesRequest}
+ */
+public class ListSnapshotFilesResponse
+{
+    private final List<FileInfo> snapshotFilesInfo;
+
+    public ListSnapshotFilesResponse()
+    {
+        this.snapshotFilesInfo = new ArrayList<>();
+    }
+
+    public void addSnapshotFile(FileInfo fileInfo)
+    {
+        snapshotFilesInfo.add(fileInfo);
+    }
+
+    public List<FileInfo> getSnapshotFilesInfo()
+    {
+        return snapshotFilesInfo;
+    }
+
+    public static class FileInfo
+    {
+        public final long size;
+        public final String host;
+        public final int port;
+        public final int dataDirIndex;
+        public final String snapshotName;
+        public final String keySpaceName;
+        public final String tableName;
+        public final String fileName;
+
+        public FileInfo(@JsonProperty("size") long size,
+                        @JsonProperty("host") String host,
+                        @JsonProperty("port") int port,
+                        @JsonProperty("dataDirIndex") int dataDirIndex,
+                        @JsonProperty("snapshotName") String snapshotName,
+                        @JsonProperty("keySpaceName") String keySpaceName,
+                        @JsonProperty("tableName") String tableName,
+                        @JsonProperty("fileName") String fileName)
+        {
+            this.size = size;
+            this.host = host;
+            this.port = port;
+            this.dataDirIndex = dataDirIndex;
+            this.snapshotName = snapshotName;
+            this.keySpaceName = keySpaceName;
+            this.tableName = tableName;
+            this.fileName = fileName;
+        }
+
+        public String ssTableComponentPath()
+        {
+            return Paths.get(keySpaceName, tableName, fileName).toString();
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            FileInfo fileInfo = (FileInfo) o;
+            return size == fileInfo.size &&
+                   port == fileInfo.port &&
+                   dataDirIndex == fileInfo.dataDirIndex &&
+                   Objects.equals(host, fileInfo.host) &&
+                   Objects.equals(snapshotName, fileInfo.snapshotName) &&
+                   Objects.equals(keySpaceName, fileInfo.keySpaceName) &&
+                   Objects.equals(tableName, fileInfo.tableName) &&
+                   Objects.equals(fileName, fileInfo.fileName);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(size, host, port, dataDirIndex, snapshotName, keySpaceName, tableName, fileName);
+        }
+    }
+
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ListSnapshotFilesResponse that = (ListSnapshotFilesResponse) o;
+        return Objects.equals(snapshotFilesInfo, that.snapshotFilesInfo);
+    }
+
+    public int hashCode()
+    {
+        return Objects.hash(snapshotFilesInfo);
+    }
+}

Review Comment:
   nit: add tailing empty line.



##########
src/main/java/org/apache/cassandra/sidecar/MainModule.java:
##########
@@ -152,17 +162,19 @@ public Router vertxRouter(Vertx vertx,
 
         // add custom routers
         final String componentRoute = "/keyspace/:keyspace/table/:table/snapshots/:snapshot/component/:component";
-        final String defaultStreamRoute = V1_API_VERSION + componentRoute;
-        final String instanceSpecificStreamRoute = V1_API_VERSION + "/instance/:instanceId" + componentRoute;
-        router.route().method(HttpMethod.GET)
-              .path(defaultStreamRoute)
-              .handler(streamSSTableComponentHandler::handleAllRequests)
+        final String defaultStreamRoute = API_V1_VERSION + componentRoute;
+        router.get(defaultStreamRoute)
+              .handler(streamSSTableComponentHandler)
               .handler(fileStreamHandler);
 
-        router.route().method(HttpMethod.GET)
-              .path(instanceSpecificStreamRoute)
-              .handler(streamSSTableComponentHandler::handlePerInstanceRequests)
-              .handler(fileStreamHandler);
+        final String listSnapshotFilesRoute1 = "/keyspace/:keyspace/table/:table/snapshots/:snapshot";
+        final String listSnapshotFilesRoute2 = "/snapshots/:snapshot";
+

Review Comment:
   1 and 2... Can we pick a more descriptive name? How about `listSnapshotFilesWithQualifiedTableNameRoute` and `listSnapshotFilesRoute`



##########
src/main/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandler.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.cassandra.sidecar.routes;
+
+import java.io.FileNotFoundException;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.CompositeFuture;
+import io.vertx.core.Future;
+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.common.data.ListSnapshotFilesRequest;
+import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesResponse;
+import org.apache.cassandra.sidecar.snapshots.SnapshotPathBuilder;
+
+/**
+ * ListSnapshotFilesHandler class lists paths of all the snapshot files of a given snapshot name.
+ * Query param includeSecondaryIndexFiles is used to request secondary index files along with other files
+ * 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
+ * <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
+ */
+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 final SnapshotPathBuilder builder;
+
+    @Inject
+    public ListSnapshotFilesHandler(SnapshotPathBuilder builder, InstancesConfig instancesConfig)
+    {
+        super(instancesConfig);
+        this.builder = builder;
+    }
+
+    @Override
+    public void handle(RoutingContext context)
+    {
+        final HttpServerRequest request = context.request();
+        final String host = getHost(context);
+        final SocketAddress remoteAddress = request.remoteAddress();
+        final ListSnapshotFilesRequest requestParams = extractParamsOrThrow(context);
+        logger.debug("ListSnapshotFilesHandler received request: {} from: {}. Instance: {}",
+                     requestParams, remoteAddress, host);
+
+        Future<List<Pair<String, FileProps>>> future;
+        boolean secondaryIndexFiles = requestParams.includeSecondaryIndexFiles();
+        if (requestParams.getKeyspace() != null && requestParams.getTableName() != null)
+        {
+            future = builder.build(host, requestParams)
+                            .compose(directory -> builder.listSnapshotDirectory(directory, secondaryIndexFiles));
+        }
+        else
+        {
+            //noinspection unchecked
+            future = builder.findSnapshotDirectories(host, requestParams.getSnapshotName())
+                            .compose(snapshotDirectoryList ->
+                            {
+                                //noinspection rawtypes
+                                List<Future> futures =
+                                snapshotDirectoryList.stream()
+                                                     .map(directory ->
+                                                          builder.listSnapshotDirectory(directory, secondaryIndexFiles))
+                                                     .collect(Collectors.toList());
+
+                                return CompositeFuture.all(futures);
+                            })
+                            .map(ar -> ar.list()
+                                         .stream()
+                                         .flatMap(l -> ((List<Pair<String, FileProps>>) l).stream())
+                                         .collect(Collectors.toList()));
+        }
+
+        future.onSuccess(fileList ->
+               {
+                  if (fileList.isEmpty())
+                  {
+                      context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(),
+                                                     "Snapshot '" + requestParams.getSnapshotName() + "' not found"));
+                  }
+                  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));
+                  }
+              });
+    }
+
+    private ListSnapshotFilesResponse buildResponse(String host,
+                                                    ListSnapshotFilesRequest request,
+                                                    List<Pair<String, FileProps>> 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();
+
+        for (Pair<String, FileProps> file : fileList)
+        {
+            Path pathFromDataDir = dataDirPath.relativize(Paths.get(file.getLeft()));
+
+            String keyspace = request.getKeyspace() != null
+                              ? request.getKeyspace()
+                              : pathFromDataDir.getName(0).toString();
+            // table name might include a dash (-) with the table UUID so we always use it as part of the response
+            String tableName = pathFromDataDir.getName(1).toString();
+            String fileName = pathFromDataDir.subpath(4, pathFromDataDir.getNameCount()).toString();
+
+            response.addSnapshotFile(new ListSnapshotFilesResponse.FileInfo(file.getRight().size(),
+                                                                            host,
+                                                                            sidecarPort,
+                                                                            DATA_DIR_INDEX,
+                                                                            snapshotName,
+                                                                            keyspace,
+                                                                            tableName,
+                                                                            fileName));
+        }
+        return response;
+    }
+
+    private ListSnapshotFilesRequest extractParamsOrThrow(final RoutingContext context)
+    {
+        boolean includeSecondaryIndexFiles =
+        "true".equalsIgnoreCase(context.request().getParam(INCLUDE_SECONDARY_INDEX_FILES, "false"));
+
+        return new ListSnapshotFilesRequest(context.pathParam("keyspace"),
+                                            context.pathParam("table"),
+                                            context.pathParam("snapshot"),

Review Comment:
   weak nit: those url segment names can be constants. And the handler can define the routes to be referenced in the main module. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/ListSnapshotFilesResponse.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.cassandra.sidecar.common.data;
+
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A class representing a response for the {@link ListSnapshotFilesRequest}
+ */
+public class ListSnapshotFilesResponse

Review Comment:
   It should be in `package org.apache.cassandra.sidecar.models`. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java:
##########
@@ -156,7 +156,7 @@ public synchronized void healthCheck()
                 adapter = versionProvider.getCassandra(version).create(cqlSession);
                 logger.info("Cassandra version change detected. New adapter loaded: {}", adapter);
             }
-            logger.info("Cassandra version {}", version);
+            logger.debug("Cassandra version {}", version);

Review Comment:
   Why changing the log level?



##########
common/src/main/java/org/apache/cassandra/sidecar/common/utils/ValidationConfiguration.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.sidecar.common.utils;
+
+import java.util.Set;
+
+/**
+ * An interface to provide validation configuration parameters
+ */
+public interface ValidationConfiguration

Review Comment:
   The only useful thing by extracting out the interface is to adapt to different cassandra versions. But it does not seems to be there are any difference with the current versions. So it only adds complexity. Are there others that I was missing? 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/utils/ValidationConfiguration.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.sidecar.common.utils;
+
+import java.util.Set;
+
+/**
+ * An interface to provide validation configuration parameters
+ */
+public interface ValidationConfiguration
+{
+    /**
+     * @return a set of forbidden directories
+     */
+    Set<String> getForbiddenDirs();
+
+    /**
+     * @return a patter for allowed characters

Review Comment:
   1. typo, `pattern`
   2. The name is vague to understand the purpose. Maybe `getCharsAllowedPatternForComponentName`



##########
common/src/test/java/org/apache/cassandra/sidecar/common/CassandraInputValidatorTest.java:
##########
@@ -133,11 +160,11 @@ public void testValidateFileName_withoutFileName_expectException()
     @Test
     public void testValidateSnapshotName_validSnapshotNames_expectNoException()
     {
-        ValidationUtils.validateSnapshotName("valid-snapshot-name");
-        ValidationUtils.validateSnapshotName("valid\\snapshot\\name"); // Is this really valid ??
-        ValidationUtils.validateSnapshotName("valid:snapshot:name");
-        ValidationUtils.validateSnapshotName("valid$snapshot$name");
-        ValidationUtils.validateSnapshotName("valid snapshot name");
+        instance.validateSnapshotName("valid-snapshot-name");
+        instance.validateSnapshotName("valid\\snapshot\\name"); // Is this really valid ??

Review Comment:
   It is invalid for windows. For unix it is OK. Can we remove the question comment? 



##########
src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cassandra.sidecar.routes;
+
+import org.apache.commons.lang3.StringUtils;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.Handler;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.handler.HttpException;
+import org.apache.cassandra.sidecar.cluster.InstancesConfig;
+
+import static org.apache.cassandra.sidecar.utils.RequestUtils.extractHostAddressWithoutPort;
+
+/**
+ * An abstract {@link Handler<RoutingContext>} that provides common functionality for handler
+ * implementations.
+ */
+public abstract class AbstractHandler implements Handler<RoutingContext>
+{
+    protected static final String INSTANCE_ID = "instanceId";
+
+    protected final InstancesConfig instancesConfig;
+
+    /**
+     * Constructs a handler with the provided {@code instancesConfig}
+     *
+     * @param instancesConfig the instances configuration
+     */
+    protected AbstractHandler(InstancesConfig instancesConfig)
+    {
+        this.instancesConfig = instancesConfig;
+    }
+
+    /**
+     * Returns the host from the path if the requests contains the {@code /instance/} path parameter,
+     * otherwise it returns the host parsed from the request.
+     *
+     * @param context the routing context
+     * @return the host for the routing context
+     * @throws HttpException when the {@code /instance/} path parameter is {@code null}
+     */
+    public String getHost(RoutingContext context)
+    {
+        if (context.request().params().contains(INSTANCE_ID))
+        {
+            String instanceIdParam = context.request().getParam(INSTANCE_ID);
+            if (instanceIdParam == null)
+            {
+                throw new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
+                                        "InstanceId query parameter must be provided");
+            }
+
+            if (!StringUtils.isNumeric(instanceIdParam))
+            {
+                throw new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
+                                        "InstanceId query parameter must be a valid integer");
+            }
+
+            int instanceId = Integer.parseInt(instanceIdParam);

Review Comment:
   I think we can combine the validation and parsing, using the parsing result as the source of truth to avoid any mismatching logic between `isNumeric` and `parseInt`.
   
   ```suggestion
               try
               {
                   int instanceId = Integer.parseInt(instanceIdParam);
                   return instancesConfig.instanceFromId(instanceId).host();
               }
               catch (NumberFormatException ex)
               {
                   throw new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
                                           "InstanceId query parameter must be a valid integer");
               }
   ```



##########
src/main/java/org/apache/cassandra/sidecar/routes/CassandraHealthService.java:
##########
@@ -64,10 +65,17 @@ public CassandraHealthService(InstanceMetadataFetcher metadataFetcher)
     })
     @GET
     @Path("/v1/cassandra/__health")
-    public Response getCassandraHealth(@Context HttpServerRequest req)
+    public Response getCassandraHealth(@Context HttpServerRequest req, @QueryParam("instanceId") Integer instanceId)

Review Comment:
   `@QueryParam(AbstractHandler.INSTANCE_ID)`



##########
common/src/main/java/org/apache/cassandra/sidecar/common/utils/ValidationConfiguration.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.sidecar.common.utils;
+
+import java.util.Set;
+
+/**
+ * An interface to provide validation configuration parameters
+ */
+public interface ValidationConfiguration

Review Comment:
   The refactor does not seem to be required by this patch too. 



##########
src/main/java/org/apache/cassandra/sidecar/routes/ListSnapshotFilesHandler.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.cassandra.sidecar.routes;
+
+import java.io.FileNotFoundException;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.CompositeFuture;
+import io.vertx.core.Future;
+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.common.data.ListSnapshotFilesRequest;
+import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesResponse;
+import org.apache.cassandra.sidecar.snapshots.SnapshotPathBuilder;
+
+/**
+ * ListSnapshotFilesHandler class lists paths of all the snapshot files of a given snapshot name.
+ * Query param includeSecondaryIndexFiles is used to request secondary index files along with other files
+ * 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
+ * <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
+ */
+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 final SnapshotPathBuilder builder;
+
+    @Inject
+    public ListSnapshotFilesHandler(SnapshotPathBuilder builder, InstancesConfig instancesConfig)
+    {
+        super(instancesConfig);
+        this.builder = builder;
+    }
+
+    @Override
+    public void handle(RoutingContext context)
+    {
+        final HttpServerRequest request = context.request();
+        final String host = getHost(context);
+        final SocketAddress remoteAddress = request.remoteAddress();
+        final ListSnapshotFilesRequest requestParams = extractParamsOrThrow(context);
+        logger.debug("ListSnapshotFilesHandler received request: {} from: {}. Instance: {}",
+                     requestParams, remoteAddress, host);
+
+        Future<List<Pair<String, FileProps>>> future;
+        boolean secondaryIndexFiles = requestParams.includeSecondaryIndexFiles();
+        if (requestParams.getKeyspace() != null && requestParams.getTableName() != null)
+        {
+            future = builder.build(host, requestParams)
+                            .compose(directory -> builder.listSnapshotDirectory(directory, secondaryIndexFiles));
+        }
+        else
+        {
+            //noinspection unchecked
+            future = builder.findSnapshotDirectories(host, requestParams.getSnapshotName())
+                            .compose(snapshotDirectoryList ->
+                            {
+                                //noinspection rawtypes
+                                List<Future> futures =
+                                snapshotDirectoryList.stream()
+                                                     .map(directory ->
+                                                          builder.listSnapshotDirectory(directory, secondaryIndexFiles))
+                                                     .collect(Collectors.toList());
+
+                                return CompositeFuture.all(futures);
+                            })
+                            .map(ar -> ar.list()
+                                         .stream()
+                                         .flatMap(l -> ((List<Pair<String, FileProps>>) l).stream())
+                                         .collect(Collectors.toList()));
+        }
+
+        future.onSuccess(fileList ->
+               {
+                  if (fileList.isEmpty())
+                  {
+                      context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(),
+                                                     "Snapshot '" + requestParams.getSnapshotName() + "' not found"));
+                  }
+                  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));
+                  }
+              });
+    }
+
+    private ListSnapshotFilesResponse buildResponse(String host,
+                                                    ListSnapshotFilesRequest request,
+                                                    List<Pair<String, FileProps>> 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();
+
+        for (Pair<String, FileProps> file : fileList)
+        {
+            Path pathFromDataDir = dataDirPath.relativize(Paths.get(file.getLeft()));
+
+            String keyspace = request.getKeyspace() != null
+                              ? request.getKeyspace()
+                              : pathFromDataDir.getName(0).toString();
+            // table name might include a dash (-) with the table UUID so we always use it as part of the response
+            String tableName = pathFromDataDir.getName(1).toString();
+            String fileName = pathFromDataDir.subpath(4, pathFromDataDir.getNameCount()).toString();
+

Review Comment:
   For readability, let's define constants to name the positions...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org