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]