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

frankgh pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-sidecar.git


The following commit(s) were added to refs/heads/trunk by this push:
     new a2bf68a0 CASSSIDECAR-388: Fixing Clear Snapshot fails when 
table/keyspace has mixed case in the name (#296)
a2bf68a0 is described below

commit a2bf68a02be001f285190ed7a98cfb20b88b5759
Author: Shruti Sekaran <[email protected]>
AuthorDate: Fri Dec 5 06:58:13 2025 -0800

    CASSSIDECAR-388: Fixing Clear Snapshot fails when table/keyspace has mixed 
case in the name (#296)
    
    Patch by Shruti Sekaran; reviewed by Yifan Cai, Francisco Guerrero for 
CASSSIDECAR-388
---
 CHANGES.txt                                        |   1 +
 .../testing/SharedClusterIntegrationTestBase.java  |  44 ++++++
 .../routes/ClearSnapshotIntegrationTest.java       | 166 +++++++++++++++++++++
 .../validations/ValidateTableExistenceHandler.java |   4 +-
 .../ClearSnapshotHandlerIntegrationTest.java       | 131 ----------------
 5 files changed, 213 insertions(+), 133 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index f2a549c1..d173bd98 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 0.3.0
 -----
+ * Adding support for quoted tables and keyspaces in snapshot cleanup 
(CASSSIDECAR-388)
  * File descriptor leak after file streamed in Sidecar Client (CASSSIDECAR-386)
  * Endpoint to invalidate auth caches as required (CASSSIDECAR-364)
  * Add /api/v2/cassandra/settings which will return Cassandra configurations 
stored in system_views.settings (CASSSIDECAR-272)
diff --git 
a/integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/SharedClusterIntegrationTestBase.java
 
b/integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/SharedClusterIntegrationTestBase.java
index f766ed3a..23b28d2c 100644
--- 
a/integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/SharedClusterIntegrationTestBase.java
+++ 
b/integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/SharedClusterIntegrationTestBase.java
@@ -19,10 +19,12 @@
 
 package org.apache.cassandra.sidecar.testing;
 
+import java.io.IOException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.UnknownHostException;
+import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Collections;
@@ -38,6 +40,7 @@ import java.util.concurrent.TimeoutException;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
 import com.google.common.util.concurrent.Uninterruptibles;
@@ -612,6 +615,47 @@ public abstract class SharedClusterIntegrationTestBase
         return builder.build();
     }
 
+    /**
+     * Recursively searches for files or directories matching the target name 
within the given keyspace.
+     * Note: must disable compaction, otherwise the file tree can be mutated 
while walking and test becomes flaky.
+     * Append WITH_COMPACTION_DISABLED to the table create statement.
+     *
+     * @param hostname     the hostname of the instance
+     * @param keyspaceName the keyspace name
+     * @param target       the target file or directory name to find
+     * @return a list of paths that match the target
+     */
+    protected List<Path> findChildFile(String hostname, String keyspaceName, 
String target)
+    {
+        InstanceMetadata instance = 
serverWrapper.injector.getInstance(InstancesMetadata.class).instanceFromHost(hostname);
+        List<String> dataDirs = instance.dataDirs();
+
+        return dataDirs.stream()
+                       .flatMap(dir -> findChildFile(Paths.get(dir, 
keyspaceName), target).stream())
+                       .collect(Collectors.toList());
+    }
+
+    /**
+     * Recursively searches for files or directories matching the target name 
within the given path.
+     *
+     * @param path   the root path to search from
+     * @param target the target file or directory name to find
+     * @return a list of paths that match the target
+     */
+    private List<Path> findChildFile(Path path, String target)
+    {
+        try (Stream<Path> walkStream = Files.walk(path))
+        {
+            return walkStream.filter(p -> p.toString().endsWith(target)
+                                          || p.toString().contains("/" + 
target + "/"))
+                             .collect(Collectors.toList());
+        }
+        catch (IOException e)
+        {
+            return Collections.emptyList();
+        }
+    }
+
     /**
      * Wraps the Sidecar server and keeps a reference to the injector to be 
able to dynamically retrieve
      * objects for testing purposes
diff --git 
a/integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/ClearSnapshotIntegrationTest.java
 
b/integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/ClearSnapshotIntegrationTest.java
new file mode 100644
index 00000000..a8d198f8
--- /dev/null
+++ 
b/integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/ClearSnapshotIntegrationTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.nio.file.Path;
+import java.util.List;
+import java.util.UUID;
+
+import org.junit.jupiter.api.Test;
+
+import io.vertx.core.buffer.Buffer;
+import io.vertx.core.http.HttpResponseExpectation;
+import io.vertx.ext.web.client.HttpResponse;
+import org.apache.cassandra.sidecar.testing.QualifiedName;
+import 
org.apache.cassandra.sidecar.testing.SharedClusterSidecarIntegrationTestBase;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.cassandra.testing.TestUtils.DC1_RF1;
+import static org.apache.cassandra.testing.utils.AssertionUtils.getBlocking;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration tests for clear snapshot endpoint
+ */
+class ClearSnapshotIntegrationTest extends 
SharedClusterSidecarIntegrationTestBase
+{
+    private static final String SNAPSHOT_ROUTE_TEMPLATE = 
"/api/v1/keyspaces/%s/tables/%s/snapshots/%s";
+
+    @Test
+    void deleteSnapshotFailsWhenKeyspaceDoesNotExist()
+    {
+        String testRoute = String.format(SNAPSHOT_ROUTE_TEMPLATE, 
"non_existent", "testtable", "my-snapshot");
+        assertNotFoundOnDeleteSnapshot(testRoute);
+    }
+
+    @Test
+    void deleteSnapshotFailsWhenTableDoesNotExist()
+    {
+        String testRoute = String.format(SNAPSHOT_ROUTE_TEMPLATE, 
"testkeyspace", "non_existent", "my-snapshot");
+        assertNotFoundOnDeleteSnapshot(testRoute);
+    }
+
+    @Test
+    void testDeleteSnapshotEndpoint()
+    {
+        testSnapshotCreateAndDelete(new QualifiedName("testkeyspace", 
"testtable"));
+    }
+
+    @Test
+    void testDeleteSnapshotWhenQuotedTableExists()
+    {
+        testSnapshotCreateAndDelete(new QualifiedName("testkeyspace", 
"QuotedTable", false, true));
+    }
+
+    @Test
+    void testDeleteSnapshotWhenQuotedKeyspaceExists()
+    {
+        testSnapshotCreateAndDelete(new QualifiedName("QuotedKeyspace", 
"testtable", true, false));
+    }
+
+    @Test
+    void testDeleteSnapshotWhenBothKeyspaceAndTableAreQuoted()
+    {
+        testSnapshotCreateAndDelete(new QualifiedName("QuotedKeyspace", 
"QuotedTable", true, true));
+    }
+
+    private void testSnapshotCreateAndDelete(QualifiedName tableName)
+    {
+        String snapshotName = "my-snapshot-" + UUID.randomUUID();
+        String testRoute = String.format(SNAPSHOT_ROUTE_TEMPLATE,
+                                         tableName.maybeQuotedKeyspace(),
+                                         tableName.maybeQuotedTable(),
+                                         snapshotName);
+
+        // Create the snapshot
+        HttpResponse<Buffer> createResponse = getBlocking(
+        trustedClient().put(serverWrapper.serverPort, "127.0.0.1", testRoute)
+                       .send());
+        assertThat(createResponse.statusCode()).isEqualTo(OK.code());
+
+        // Verify snapshot files exist
+        List<Path> snapshotFiles = findChildFile("127.0.0.1", 
tableName.keyspace(), snapshotName);
+        assertThat(snapshotFiles).isNotEmpty();
+
+        // Delete the snapshot
+        HttpResponse<Buffer> deleteResponse = getBlocking(
+        trustedClient().delete(serverWrapper.serverPort, "127.0.0.1", 
testRoute)
+                       .send());
+        assertThat(deleteResponse.statusCode()).isEqualTo(OK.code());
+
+        // Verify snapshot files were deleted
+        List<Path> snapshotFilesAfterDelete = findChildFile("127.0.0.1", 
tableName.keyspace(), snapshotName);
+        assertThat(snapshotFilesAfterDelete).isEmpty();
+    }
+
+    private void assertNotFoundOnDeleteSnapshot(String testRoute)
+    {
+        HttpResponse<Buffer> response = 
getBlocking(trustedClient().delete(serverWrapper.serverPort, "127.0.0.1", 
testRoute)
+                                                                   .send()
+                                                                   
.expecting(HttpResponseExpectation.SC_NOT_FOUND));
+        assertThat(response.statusCode()).isEqualTo(NOT_FOUND.code());
+    }
+
+    @Override
+    protected void initializeSchemaForTest()
+    {
+        createTestKeyspace("testkeyspace", DC1_RF1);
+        createTestKeyspace("\"QuotedKeyspace\"", DC1_RF1);
+
+        createTestTable(new QualifiedName("testkeyspace", "testtable"),
+                        "CREATE TABLE %s (id text PRIMARY KEY, name text)");
+        createTestTable(new QualifiedName("testkeyspace", "QuotedTable", 
false, true),
+                        "CREATE TABLE %s (id text PRIMARY KEY, name text)");
+
+        createTestTable(new QualifiedName("QuotedKeyspace", "testtable", true, 
false),
+                        "CREATE TABLE %s (id text PRIMARY KEY, name text)");
+        createTestTable(new QualifiedName("QuotedKeyspace", "QuotedTable", 
true, true),
+                        "CREATE TABLE %s (id text PRIMARY KEY, name text)");
+
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO testkeyspace.testtable (id, name) VALUES ('1', 
'Francisco')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO testkeyspace.testtable (id, name) VALUES ('2', 
'Saranya')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO testkeyspace.testtable (id, name) VALUES ('3', 'Yifan')");
+
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO testkeyspace.\"QuotedTable\" (id, name) VALUES ('1', 
'Francisco')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO testkeyspace.\"QuotedTable\" (id, name) VALUES ('2', 
'Saranya')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO testkeyspace.\"QuotedTable\" (id, name) VALUES ('3', 
'Yifan')");
+
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO \"QuotedKeyspace\".testtable (id, name) VALUES ('1', 
'Francisco')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO \"QuotedKeyspace\".testtable (id, name) VALUES ('2', 
'Saranya')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO \"QuotedKeyspace\".testtable (id, name) VALUES ('3', 
'Yifan')");
+
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO \"QuotedKeyspace\".\"QuotedTable\" (id, name) VALUES 
('1', 'Francisco')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO \"QuotedKeyspace\".\"QuotedTable\" (id, name) VALUES 
('2', 'Saranya')");
+        cluster.schemaChangeIgnoringStoppedInstances(
+        "INSERT INTO \"QuotedKeyspace\".\"QuotedTable\" (id, name) VALUES 
('3', 'Yifan')");
+    }
+}
diff --git 
a/server/src/main/java/org/apache/cassandra/sidecar/handlers/validations/ValidateTableExistenceHandler.java
 
b/server/src/main/java/org/apache/cassandra/sidecar/handlers/validations/ValidateTableExistenceHandler.java
index 83abccc7..4c089481 100644
--- 
a/server/src/main/java/org/apache/cassandra/sidecar/handlers/validations/ValidateTableExistenceHandler.java
+++ 
b/server/src/main/java/org/apache/cassandra/sidecar/handlers/validations/ValidateTableExistenceHandler.java
@@ -76,7 +76,7 @@ public class ValidateTableExistenceHandler extends 
AbstractHandler<QualifiedTabl
             return;
         }
 
-        getKeyspaceMetadata(host, input.keyspace())
+        getKeyspaceMetadata(host, input.maybeQuotedKeyspace())
         .onFailure(context::fail) // fail the request with the internal server 
error thrown from getKeyspaceMetadata
         .onSuccess(keyspaceMetadata -> {
             if (keyspaceMetadata == null)
@@ -88,7 +88,7 @@ public class ValidateTableExistenceHandler extends 
AbstractHandler<QualifiedTabl
 
             RoutingContextUtils.put(context, 
RoutingContextUtils.SC_KEYSPACE_METADATA, keyspaceMetadata);
 
-            String table = input.tableName();
+            String table = input.maybeQuotedTableName();
             if (table == null)
             {
                 context.next();
diff --git 
a/server/src/test/integration/org/apache/cassandra/sidecar/routes/snapshots/ClearSnapshotHandlerIntegrationTest.java
 
b/server/src/test/integration/org/apache/cassandra/sidecar/routes/snapshots/ClearSnapshotHandlerIntegrationTest.java
deleted file mode 100644
index 8ec190d5..00000000
--- 
a/server/src/test/integration/org/apache/cassandra/sidecar/routes/snapshots/ClearSnapshotHandlerIntegrationTest.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.sidecar.routes.snapshots;
-
-import java.nio.file.Path;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.jupiter.api.extension.ExtendWith;
-
-import com.datastax.driver.core.Session;
-import io.vertx.ext.web.client.WebClient;
-import io.vertx.ext.web.client.predicate.ResponsePredicate;
-import io.vertx.junit5.VertxExtension;
-import io.vertx.junit5.VertxTestContext;
-import org.apache.cassandra.sidecar.common.server.data.QualifiedTableName;
-import org.apache.cassandra.sidecar.testing.IntegrationTestBase;
-import org.apache.cassandra.testing.CassandraIntegrationTest;
-
-import static io.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.assertj.core.api.Assertions.assertThat;
-
-@ExtendWith(VertxExtension.class)
-class ClearSnapshotHandlerIntegrationTest extends IntegrationTestBase
-{
-    @CassandraIntegrationTest
-    void deleteSnapshotFailsWhenKeyspaceDoesNotExist(VertxTestContext context) 
throws Exception
-    {
-        String testRoute = 
"/api/v1/keyspaces/non_existent/tables/testtable/snapshots/my-snapshot";
-        assertNotFoundOnDeleteSnapshot(context, testRoute);
-    }
-
-    @CassandraIntegrationTest
-    void deleteSnapshotFailsWhenTableDoesNotExist(VertxTestContext context)
-    throws Exception
-    {
-        createTestKeyspace();
-        createTestTableAndPopulate();
-
-        String testRoute = 
"/api/v1/keyspaces/testkeyspace/tables/non_existent/snapshots/my-snapshot";
-        assertNotFoundOnDeleteSnapshot(context, testRoute);
-    }
-
-    @CassandraIntegrationTest(numDataDirsPerInstance = 1)
-        // Set to > 1 to fail test
-    void testDeleteSnapshotEndpoint(VertxTestContext context)
-    throws Exception
-    {
-        createTestKeyspace();
-        QualifiedTableName tableName = createTestTableAndPopulate();
-
-        WebClient client = mTLSClient();
-        String snapshotName = "my-snapshot" + UUID.randomUUID();
-        String testRoute = 
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/%s",
-                                         tableName.maybeQuotedKeyspace(), 
tableName.maybeQuotedTableName(),
-                                         snapshotName);
-
-        // first create the snapshot
-        client.put(server.actualPort(), "127.0.0.1", testRoute)
-              .expect(ResponsePredicate.SC_OK)
-              .send(context.succeeding(
-              createResponse ->
-              context.verify(() -> {
-                  assertThat(createResponse.statusCode()).isEqualTo(OK.code());
-                  final List<Path> found =
-                  findChildFile(sidecarTestContext, "127.0.0.1", 
tableName.keyspace(), snapshotName);
-                  assertThat(found).isNotEmpty();
-
-                  // snapshot directory exists inside data directory
-                  assertThat(found).isNotEmpty();
-
-                  // then delete the snapshot
-                  client.delete(server.actualPort(), "127.0.0.1", testRoute)
-                        .expect(ResponsePredicate.SC_OK)
-                        .send(context.succeeding(
-                        deleteResponse ->
-                        context.verify(() ->
-                                       {
-                                           
assertThat(deleteResponse.statusCode()).isEqualTo(OK.code());
-                                           final List<Path> found2 =
-                                           findChildFile(sidecarTestContext,
-                                                         "127.0.0.1",
-                                                         tableName.keyspace(),
-                                                         snapshotName);
-                                           assertThat(found2).isEmpty();
-                                           context.completeNow();
-                                       })));
-              })));
-        // wait until the test completes
-        assertThat(context.awaitCompletion(30, TimeUnit.SECONDS)).isTrue();
-    }
-
-    private QualifiedTableName createTestTableAndPopulate()
-    {
-        QualifiedTableName tableName = createTestTable(
-        "CREATE TABLE %s (id text PRIMARY KEY, name text)" + 
WITH_COMPACTION_DISABLED + ";");
-        Session session = maybeGetSession();
-
-        session.execute("INSERT INTO " + tableName + " (id, name) VALUES ('1', 
'Francisco');");
-        session.execute("INSERT INTO " + tableName + " (id, name) VALUES ('2', 
'Saranya');");
-        session.execute("INSERT INTO " + tableName + " (id, name) VALUES ('3', 
'Yifan');");
-        return tableName;
-    }
-
-    private void assertNotFoundOnDeleteSnapshot(VertxTestContext context, 
String testRoute) throws Exception
-    {
-        WebClient client = mTLSClient();
-        client.delete(server.actualPort(), "127.0.0.1", testRoute)
-              .expect(ResponsePredicate.SC_NOT_FOUND)
-              .send(context.succeedingThenComplete());
-        // wait until test completes
-        assertThat(context.awaitCompletion(30, TimeUnit.SECONDS)).isTrue();
-    }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to