This is an automated email from the ASF dual-hosted git repository.
ycai 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 31fa33f CASSANDRASC-76: Sidecar does not handle keyspaces and table
names with mixed case
31fa33f is described below
commit 31fa33fcb446e522947f899d948de4042be04c62
Author: Francisco Guerrero <[email protected]>
AuthorDate: Tue Nov 14 14:42:17 2023 -0800
CASSANDRASC-76: Sidecar does not handle keyspaces and table names with
mixed case
Cassandra Sidecar does not properly handle API requests for endpoints that
have keypaces
or table names that need to be quoted, for example when names have mixed
case or when
the name is a reserved keyword in Cassandra.
In this commit, we perform special handling when the keyspace or table
names are quoted
in the path params. We add tests to ensure that handling is correct.
Additionally, we fix the validation for keyspaces and table names without
quotes and add
special validation for quoted names.
Patch by Francisco Guerrero; Reviewed by Yifan Cai for CASSANDRASC-76
---
.circleci/config.yml | 20 ++--
CHANGES.txt | 1 +
.../adapters/base/CassandraStorageOperations.java | 5 +-
.../sidecar/adapters/base/RingProvider.java | 5 +-
.../adapters/base/TokenRangeReplicaProvider.java | 7 +-
.../base/TokenRangeReplicaProviderTest.java | 50 ++++----
build.gradle | 2 +
.../client/request/TokenRangeReplicasRequest.java | 2 +
.../cassandra/sidecar/common/NodeSettings.java | 14 ++-
.../sidecar/common/StorageOperations.java | 8 +-
.../apache/cassandra/sidecar/common/data/Name.java | 106 +++++++++++++++++
.../sidecar/common/data/QualifiedTableName.java | 64 ++++++++--
.../sidecar/common/data/SSTableUploads.java | 8 +-
.../common/data/TokenRangeReplicasRequest.java | 6 +-
src/main/dist/conf/sidecar.yaml | 3 +-
.../CassandraInputValidationConfiguration.java | 14 ++-
.../CassandraInputValidationConfigurationImpl.java | 47 +++++---
.../apache/cassandra/sidecar/data/RingRequest.java | 12 +-
.../sidecar/data/SSTableImportRequest.java | 6 +-
.../sidecar/data/SSTableUploadRequest.java | 2 +-
.../cassandra/sidecar/data/SchemaRequest.java | 17 ++-
.../cassandra/sidecar/routes/AbstractHandler.java | 8 +-
.../cassandra/sidecar/routes/SchemaHandler.java | 7 +-
.../cassandra/sidecar/routes/SnapshotsHandler.java | 18 ++-
.../sstableuploads/SSTableImportHandler.java | 8 +-
.../sstableuploads/SSTableUploadHandler.java | 7 +-
.../sidecar/snapshots/SnapshotPathBuilder.java | 3 +-
.../sidecar/utils/CassandraInputValidator.java | 59 +++++++---
.../cassandra/sidecar/utils/MetadataUtils.java | 57 +++++++++
.../sidecar/utils/SSTableUploadsPathBuilder.java | 8 +-
.../cassandra/sidecar/IntegrationTestBase.java | 19 ++-
.../routes/SchemaHandlerIntegrationTest.java | 130 +++++++++++++++++++++
.../routes/SnapshotsHandlerIntegrationTest.java | 75 +++++++++---
.../org/apache/cassandra/sidecar/ThrottleTest.java | 6 +-
.../sidecar/config/SidecarConfigurationTest.java | 6 +-
.../cassandra/sidecar/data/RingRequestTest.java | 23 ++--
.../org/apache/cassandra/sidecar/mocks/V30.java | 0
.../org/apache/cassandra/sidecar/mocks/V40.java | 0
.../org/apache/cassandra/sidecar/mocks/V41.java | 0
.../sidecar/routes/SnapshotsHandlerTest.java | 16 +--
.../routes/StreamSSTableComponentHandlerTest.java | 33 +++---
.../sstableuploads/SSTableUploadHandlerTest.java | 2 +-
.../sidecar/utils/CassandraInputValidatorTest.java | 100 ++++++++--------
.../config/sidecar_multiple_instances.yaml | 3 +-
.../resources/config/sidecar_single_instance.yaml | 3 +-
src/test/resources/config/sidecar_ssl.yaml | 3 +-
.../config/sidecar_validation_configuration.yaml | 1 +
...-a9be-e428e5d7160b-Data.db => nb-1-big-Data.db} | 0
...-a9be-e428e5d7160b-Data.db => nb-1-big-Data.db} | 0
...-a9be-e428e5d7160b-Data.db => nb-1-big-Data.db} | 0
50 files changed, 750 insertions(+), 244 deletions(-)
diff --git a/.circleci/config.yml b/.circleci/config.yml
index 92f9a75..ee8fe82 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -79,8 +79,8 @@ jobs:
docker:
- image: circleci/openjdk:8-jdk-stretch
environment:
- INTEGRATION_MAX_PARALLEL_FORKS: 1
- INTEGRATION_MAX_HEAP_SIZE: "4G"
+ INTEGRATION_MAX_PARALLEL_FORKS: 2
+ INTEGRATION_MAX_HEAP_SIZE: "3500M"
resource_class: large
steps:
- checkout
@@ -99,8 +99,8 @@ jobs:
docker:
- image: circleci/openjdk:8-jdk-stretch
environment:
- INTEGRATION_MAX_PARALLEL_FORKS: 1
- INTEGRATION_MAX_HEAP_SIZE: "4G"
+ INTEGRATION_MAX_PARALLEL_FORKS: 2
+ INTEGRATION_MAX_HEAP_SIZE: "3500M"
resource_class: large
steps:
- checkout
@@ -135,8 +135,8 @@ jobs:
docker:
- image: circleci/openjdk:11-jdk-stretch
environment:
- INTEGRATION_MAX_PARALLEL_FORKS: 1
- INTEGRATION_MAX_HEAP_SIZE: "4G"
+ INTEGRATION_MAX_PARALLEL_FORKS: 2
+ INTEGRATION_MAX_HEAP_SIZE: "3500M"
resource_class: large
steps:
- checkout
@@ -155,8 +155,8 @@ jobs:
docker:
- image: circleci/openjdk:11-jdk-stretch
environment:
- INTEGRATION_MAX_PARALLEL_FORKS: 1
- INTEGRATION_MAX_HEAP_SIZE: "4G"
+ INTEGRATION_MAX_PARALLEL_FORKS: 2
+ INTEGRATION_MAX_HEAP_SIZE: "3500M"
resource_class: large
steps:
- checkout
@@ -174,8 +174,8 @@ jobs:
docker:
- image: circleci/openjdk:11-jdk-stretch
environment:
- INTEGRATION_MAX_PARALLEL_FORKS: 1
- INTEGRATION_MAX_HEAP_SIZE: "4G"
+ INTEGRATION_MAX_PARALLEL_FORKS: 2
+ INTEGRATION_MAX_HEAP_SIZE: "3500M"
resource_class: large
steps:
- checkout
diff --git a/CHANGES.txt b/CHANGES.txt
index 36aa800..f7b0572 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
1.0.0
-----
+ * Sidecar does not handle keyspaces and table names with mixed case
(CASSANDRASC-76)
* Require gossip to be enabled for ring and token ranges mapping endpoints
(CASSANDRASC-83)
* Improve TokenRangeReplicasResponse payload (CASSANDRASC-81)
* HealthCheckPeriodicTask execute never completes the promise when instances
are empty (CASSANDRASC-80)
diff --git
a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
index 58d6040..ce14d1e 100644
---
a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
+++
b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
@@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.sidecar.common.JmxClient;
import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.RingResponse;
import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
import org.apache.cassandra.sidecar.common.dns.DnsResolver;
@@ -135,7 +136,7 @@ public class CassandraStorageOperations implements
StorageOperations
* {@inheritDoc}
*/
@Override
- public RingResponse ring(@Nullable String keyspace) throws
UnknownHostException
+ public RingResponse ring(@Nullable Name keyspace) throws
UnknownHostException
{
return ringProvider.ring(keyspace);
}
@@ -144,7 +145,7 @@ public class CassandraStorageOperations implements
StorageOperations
* {@inheritDoc}
*/
@Override
- public TokenRangeReplicasResponse tokenRangeReplicas(@NotNull String
keyspace, @NotNull String partitioner)
+ public TokenRangeReplicasResponse tokenRangeReplicas(@NotNull Name
keyspace, @NotNull String partitioner)
{
return tokenRangeReplicaProvider.tokenRangeReplicas(keyspace,
Partitioner.fromClassName(partitioner));
}
diff --git
a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
index 8fd3253..0b5463b 100644
---
a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
+++
b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
@@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeStatus;
import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.RingEntry;
import org.apache.cassandra.sidecar.common.data.RingResponse;
import org.apache.cassandra.sidecar.common.dns.DnsResolver;
@@ -60,7 +61,7 @@ public class RingProvider
}
@SuppressWarnings("UnstableApiUsage")
- public RingResponse ring(@Nullable String keyspace) throws
UnknownHostException
+ public RingResponse ring(@Nullable Name keyspace) throws
UnknownHostException
{
StorageJmxOperations storageOps = initializeStorageOps();
EndpointSnitchJmxOperations epSnitchInfo = initializeEndpointProxy();
@@ -82,7 +83,7 @@ public class RingProvider
Map<String, Float> ownerships;
try
{
- ownerships = storageOps.effectiveOwnershipWithPort(keyspace);
+ ownerships = storageOps.effectiveOwnershipWithPort(keyspace !=
null ? keyspace.name() : null);
}
catch (IllegalStateException ex)
{
diff --git
a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java
b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java
index f1f5ff1..a95f6af 100644
---
a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java
+++
b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java
@@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
import org.apache.cassandra.sidecar.common.JmxClient;
import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
import
org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
import
org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaMetadata;
@@ -73,17 +74,17 @@ public class TokenRangeReplicaProvider
this.dnsResolver = dnsResolver;
}
- public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace,
Partitioner partitioner)
+ public TokenRangeReplicasResponse tokenRangeReplicas(Name keyspace,
Partitioner partitioner)
{
Objects.requireNonNull(keyspace, "keyspace must be non-null");
StorageJmxOperations storage = initializeStorageOps();
List<TokenRangeReplicas> naturalTokenRangeReplicas =
- getTokenRangeReplicas("Natural", keyspace, partitioner,
storage::getRangeToEndpointWithPortMap);
+ getTokenRangeReplicas("Natural", keyspace.name(), partitioner,
storage::getRangeToEndpointWithPortMap);
// Pending ranges include bootstrap tokens and leaving endpoints as
represented in the Cassandra TokenMetadata
List<TokenRangeReplicas> pendingTokenRangeReplicas =
- getTokenRangeReplicas("Pending", keyspace, partitioner,
storage::getPendingRangeToEndpointWithPortMap);
+ getTokenRangeReplicas("Pending", keyspace.name(), partitioner,
storage::getPendingRangeToEndpointWithPortMap);
// Merge natural and pending range replicas to generate candidates for
write-replicas
List<TokenRangeReplicas> allTokenRangeReplicas = new
ArrayList<>(naturalTokenRangeReplicas);
diff --git
a/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java
b/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java
index 8c244bb..a3f5d04 100644
---
a/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java
+++
b/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java
@@ -32,6 +32,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
import org.apache.cassandra.sidecar.common.dns.DnsResolver;
import org.assertj.core.api.InstanceOfAssertFactories;
@@ -61,7 +62,7 @@ public class TokenRangeReplicaProviderTest
private static final List<String> TEST_MULTI_DC_ENDPOINTS =
Arrays.asList("128.0.0.1:7000",
"128.0.0.2:7000",
"127.0.0.4:7000");
- public static final String TEST_KEYSPACE = "test_keyspace";
+ public static final Name TEST_KEYSPACE = new Name("test_keyspace");
private static final String TEST_HOSTNAME = "Hostname";
@@ -98,9 +99,10 @@ public class TokenRangeReplicaProviderTest
Map<List<String>, List<String>> readReplicaMappings = new HashMap<>();
readReplicaMappings.put(TOKEN_RANGE1, TEST_ENDPOINTS1);
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name())).thenReturn(readReplicaMappings);
Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(writeReplicaMappings);
when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
when(storageOperations.getLiveNodesWithPort()).thenReturn(TEST_ENDPOINTS1);
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
@@ -134,9 +136,10 @@ public class TokenRangeReplicaProviderTest
when(storageOperations.getLiveNodesWithPort()).thenReturn(allLiveNodes);
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name())).thenReturn(readReplicaMappings);
Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(writeReplicaMappings);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("128.0.0.1:7000",
"127.0.0.2:7000"));
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
when(clusterMembershipOperations.getAllEndpointStatesWithPort()).thenReturn(generateSampleGossip("NORMAL",
@@ -150,7 +153,7 @@ public class TokenRangeReplicaProviderTest
when(endpointOperations.getDatacenter(startsWith("127"))).thenReturn(TEST_DC1);
when(endpointOperations.getDatacenter(startsWith("128"))).thenReturn(TEST_DC2);
- TokenRangeReplicasResponse result =
instance.tokenRangeReplicas("test_keyspace", Partitioner.Random);
+ TokenRangeReplicasResponse result =
instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Random);
assertThat(result).isNotNull();
assertThat(result.writeReplicas().size()).isEqualTo(2);
// Single token range
@@ -187,9 +190,10 @@ public class TokenRangeReplicaProviderTest
when(storageOperations.getLiveNodesWithPort()).thenReturn(allLiveNodes);
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name())).thenReturn(readReplicaMappings);
Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(writeReplicaMappings);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.1:7000",
"127.0.0.2:7000",
"128.0.0.1:7000"));
@@ -204,7 +208,7 @@ public class TokenRangeReplicaProviderTest
when(endpointOperations.getDatacenter(startsWith("127"))).thenReturn(TEST_DC1);
when(endpointOperations.getDatacenter(startsWith("128"))).thenReturn(TEST_DC2);
- TokenRangeReplicasResponse result =
instance.tokenRangeReplicas("test_keyspace", Partitioner.Random);
+ TokenRangeReplicasResponse result =
instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Random);
assertThat(result).isNotNull();
assertThat(result.writeReplicas().size()).isEqualTo(2);
// 2 token ranges
@@ -237,8 +241,10 @@ public class TokenRangeReplicaProviderTest
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(readReplicaMappings);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(writeReplicaMappings);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.3:7000",
"128.0.0.1:7000"));
when(clusterMembershipOperations.getAllEndpointStatesWithPort())
.thenReturn(generateSampleGossip("NORMAL",
@@ -251,7 +257,7 @@ public class TokenRangeReplicaProviderTest
when(endpointOperations.getDatacenter(startsWith("127"))).thenReturn(TEST_DC1);
when(endpointOperations.getDatacenter(startsWith("128"))).thenReturn(TEST_DC2);
- TokenRangeReplicasResponse result =
instance.tokenRangeReplicas("test_keyspace", Partitioner.Random);
+ TokenRangeReplicasResponse result =
instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Random);
assertThat(result).isNotNull();
assertThat(result.writeReplicas().size()).isEqualTo(2);
assertThat(result.readReplicas().size()).isEqualTo(1);
@@ -279,8 +285,9 @@ public class TokenRangeReplicaProviderTest
when(storageOperations.getLiveNodesWithPort()).thenReturn(TEST_ENDPOINTS1);
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(rangeToEndpointWithPortMap);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
.thenReturn(pendingRangeToEndpointWithPortMap);
when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.4:7000",
"128.0.0.2:7000"));
@@ -327,8 +334,9 @@ public class TokenRangeReplicaProviderTest
when(storageOperations.getLiveNodesWithPort()).thenReturn(TEST_ENDPOINTS1);
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(rangeToEndpointWithPortMap);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
.thenReturn(pendingRangeToEndpointWithPortMap);
when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.2:7000",
"127.0.0.4:7000"));
@@ -377,8 +385,9 @@ public class TokenRangeReplicaProviderTest
.thenReturn(Arrays.asList("127.0.0.1:7000", "127.0.0.2:7000",
"127.0.0.3:7000", "127.0.0.4:7000"));
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(rangeToEndpointWithPortMap);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
.thenReturn(pendingRangeToEndpointWithPortMap);
when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.1:7000",
"128.0.0.1:7000"));
@@ -424,8 +433,9 @@ public class TokenRangeReplicaProviderTest
.thenReturn(Arrays.asList("127.0.0.1:7000", "127.0.0.2:7000",
"127.0.0.3:7000", "127.0.0.4:7000"));
when(storageOperations.getUnreachableNodesWithPort()).thenReturn(Collections.emptyList());
-
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
-
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+
when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
+ .thenReturn(rangeToEndpointWithPortMap);
+
when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE.name()))
.thenReturn(pendingRangeToEndpointWithPortMap);
when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.1:7000",
"128.0.0.1:7000"));
diff --git a/build.gradle b/build.gradle
index d733494..2e0743d 100644
--- a/build.gradle
+++ b/build.gradle
@@ -323,6 +323,8 @@ tasks.register("integrationTest", Test) {
jvmArgs(JDK11_OPTIONS)
println("JVM arguments for $project.name are $allJvmArgs")
}
+ // Disable direct memory allocator as it doesn't release properly
+ systemProperty "cassandra.netty_use_heap_allocator", "true"
systemProperty "vertxweb.environment", "dev"
// Because tests are forked, we need to explicitly pass the system
property from the
// Gradle JVM down to the children
diff --git
a/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java
b/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java
index 712ac75..91f11f7 100644
---
a/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java
+++
b/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java
@@ -29,6 +29,8 @@ public class TokenRangeReplicasRequest extends
DecodableRequest<TokenRangeReplic
{
/**
* Constructs a new request to retrieve information by keyspace from
token-range replicas endpoint
+ *
+ * @param keyspace the keyspace in Cassandra
*/
public TokenRangeReplicasRequest(String keyspace)
{
diff --git
a/common/src/main/java/org/apache/cassandra/sidecar/common/NodeSettings.java
b/common/src/main/java/org/apache/cassandra/sidecar/common/NodeSettings.java
index ef5e9c8..8833403 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/NodeSettings.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/NodeSettings.java
@@ -57,12 +57,12 @@ public class NodeSettings
* @param sidecar the settings of the Sidecar on the Cassandra
node, including its version
*/
public NodeSettings(@JsonProperty("releaseVersion") String releaseVersion,
- @JsonProperty("partitioner") String partitioner,
- @JsonProperty("sidecar") Map<String, String>
sidecar)
+ @JsonProperty("partitioner") String partitioner,
+ @JsonProperty("sidecar") Map<String, String> sidecar)
{
this.releaseVersion = releaseVersion;
- this.partitioner = partitioner;
- this.sidecar = sidecar;
+ this.partitioner = partitioner;
+ this.sidecar = sidecar;
}
@JsonProperty("releaseVersion")
@@ -91,6 +91,7 @@ public class NodeSettings
/**
* {@inheritDoc}
*/
+ @Override
public boolean equals(Object other)
{
if (this == other)
@@ -103,13 +104,14 @@ public class NodeSettings
}
NodeSettings that = (NodeSettings) other;
return Objects.equals(this.releaseVersion, that.releaseVersion)
- && Objects.equals(this.partitioner, that.partitioner)
- && Objects.equals(this.sidecar, that.sidecar);
+ && Objects.equals(this.partitioner, that.partitioner)
+ && Objects.equals(this.sidecar, that.sidecar);
}
/**
* {@inheritDoc}
*/
+ @Override
public int hashCode()
{
return Objects.hash(releaseVersion, partitioner, sidecar);
diff --git
a/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
b/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
index 75d9980..56a4de1 100644
---
a/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
+++
b/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.sidecar.common;
import java.net.UnknownHostException;
import java.util.Map;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.RingResponse;
import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
import org.jetbrains.annotations.NotNull;
@@ -58,14 +59,15 @@ public interface StorageOperations
* @return ring view
* @throws UnknownHostException when hostname of peer Cassandra nodes
cannot be resolved
*/
- RingResponse ring(@Nullable String keyspace) throws UnknownHostException;
+ RingResponse ring(@Nullable Name keyspace) throws UnknownHostException;
/**
* Get the token ranges and the corresponding read and write replicas by
datacenter
- * @param keyspace the keyspace in the Cassandra database
+ *
+ * @param keyspace the keyspace in the Cassandra database
* @param partitioner token partitioner used for token assignment
* @return token range to read and write replica mappings
*/
- TokenRangeReplicasResponse tokenRangeReplicas(@NotNull String keyspace,
+ TokenRangeReplicasResponse tokenRangeReplicas(@NotNull Name keyspace,
@NotNull String partitioner);
}
diff --git
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/Name.java
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/Name.java
new file mode 100644
index 0000000..03b1353
--- /dev/null
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/data/Name.java
@@ -0,0 +1,106 @@
+/*
+ * 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.util.Objects;
+
+/**
+ * Represents the name of keyspaces and tables defined by the grammar in
+ * <a
href="https://cassandra.apache.org/doc/4.1/cassandra/cql/ddl.html#common-definitions">Cassandra
CQL common
+ * definitions</a>
+ */
+public class Name
+{
+ private final String unquotedName;
+ private final String maybeQuotedName;
+
+ /**
+ * Constructs a {@link Name} object with the provided {@code name}.
+ *
+ * @param name the name
+ */
+ public Name(String name)
+ {
+ this(name, name);
+ }
+
+ /**
+ * Constructs a {@link Name} object with the provided {@code unquotedName}
and {@code maybeQuotedName}
+ * parameters.
+ *
+ * @param unquotedName the unquoted name
+ * @param maybeQuotedName the maybe quoted name
+ */
+ public Name(String unquotedName, String maybeQuotedName)
+ {
+ this.unquotedName = Objects.requireNonNull(unquotedName, "the unquoted
name is required");
+ this.maybeQuotedName = Objects.requireNonNull(maybeQuotedName, "the
_maybe_ quoted name is required");
+ }
+
+ /**
+ * @return the unquoted name
+ */
+ public String name()
+ {
+ return unquotedName;
+ }
+
+ /**
+ * @return the quoted name, if the original input was quoted and if the
unquoted name needs to be quoted,
+ * or the unquoted name otherwise
+ */
+ public String maybeQuotedName()
+ {
+ return maybeQuotedName;
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ Name name = (Name) o;
+ return Objects.equals(unquotedName, name.unquotedName)
+ && Objects.equals(maybeQuotedName, name.maybeQuotedName);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(unquotedName, maybeQuotedName);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public String toString()
+ {
+ return "Name{" +
+ "unquotedName='" + unquotedName + '\'' +
+ ", maybeQuotedName='" + maybeQuotedName + '\'' +
+ '}';
+ }
+}
diff --git
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/QualifiedTableName.java
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/QualifiedTableName.java
index a8517e6..984a9f3 100644
---
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/QualifiedTableName.java
+++
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/QualifiedTableName.java
@@ -19,13 +19,17 @@ package org.apache.cassandra.sidecar.common.data;
import java.util.Objects;
+import org.jetbrains.annotations.Nullable;
+
/**
* Contains the keyspace and table name in Cassandra
*/
public class QualifiedTableName
{
- private final String keyspace;
- private final String tableName;
+ @Nullable
+ private final Name keyspace;
+ @Nullable
+ private final Name table;
/**
* Constructs a qualified name with the given {@code keyspace} and {@code
tableName}
@@ -48,16 +52,42 @@ public class QualifiedTableName
*/
public QualifiedTableName(String keyspace, String tableName, boolean
required)
{
- this.keyspace = !required && keyspace == null ? null :
Objects.requireNonNull(keyspace,
-
"keyspace must not be null");
- this.tableName = !required && tableName == null ? null :
Objects.requireNonNull(tableName,
-
"tableName must not be null");
+ if (required)
+ {
+ Objects.requireNonNull(keyspace, "keyspace must not be null");
+ Objects.requireNonNull(tableName, "tableName must not be null");
+ }
+ this.keyspace = !required && keyspace == null ? null : new
Name(keyspace);
+ this.table = !required && tableName == null ? null : new
Name(tableName);
+ }
+
+ public QualifiedTableName(@Nullable Name keyspace, @Nullable Name table)
+ {
+ this.keyspace = keyspace;
+ this.table = table;
}
/**
* @return the keyspace in Cassandra
*/
public String keyspace()
+ {
+ return keyspace != null ? keyspace.name() : null;
+ }
+
+ /**
+ * @return the keyspace in Cassandra, quoted if the original input was
quoted and if
+ * the unquoted keyspace needs to be quoted
+ */
+ public String maybeQuotedKeyspace()
+ {
+ return keyspace != null ? keyspace.maybeQuotedName() : null;
+ }
+
+ /**
+ * @return the keyspace in Cassandra
+ */
+ public @Nullable Name getKeyspace()
{
return keyspace;
}
@@ -67,14 +97,32 @@ public class QualifiedTableName
*/
public String tableName()
{
- return tableName;
+ return table != null ? table.name() : null;
+ }
+
+ /**
+ * @return the table name in Cassandra, quoted if the original input was
quoted and if
+ * the unquoted table needs to be quoted
+ */
+ public String maybeQuotedTableName()
+ {
+ return table != null ? table.maybeQuotedName() : null;
+ }
+
+ /**
+ * @return the table name in Cassandra
+ */
+ public @Nullable Name table()
+ {
+ return table;
}
/**
* {@inheritDoc}
*/
+ @Override
public String toString()
{
- return keyspace + "." + tableName;
+ return maybeQuotedKeyspace() + "." + maybeQuotedTableName();
}
}
diff --git
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/SSTableUploads.java
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/SSTableUploads.java
index fc0663b..36accb2 100644
---
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/SSTableUploads.java
+++
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/SSTableUploads.java
@@ -43,17 +43,17 @@ public class SSTableUploads
/**
* @return the keyspace in Cassandra
*/
- public String keyspace()
+ public Name keyspace()
{
- return qualifiedTableName.keyspace();
+ return qualifiedTableName.getKeyspace();
}
/**
* @return the table name in Cassandra
*/
- public String tableName()
+ public Name table()
{
- return qualifiedTableName.tableName();
+ return qualifiedTableName.table();
}
/**
diff --git
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java
index e00f0ab..59e3865 100644
---
a/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java
+++
b/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java
@@ -25,14 +25,14 @@ import java.util.Objects;
*/
public class TokenRangeReplicasRequest
{
- private final String keyspace;
+ private final Name keyspace;
/**
* Constructs a ring request with {@code keyspace} parameter.
*
* @param keyspace the keyspace in Cassandra
*/
- public TokenRangeReplicasRequest(String keyspace)
+ public TokenRangeReplicasRequest(Name keyspace)
{
this.keyspace = keyspace;
}
@@ -40,7 +40,7 @@ public class TokenRangeReplicasRequest
/**
* @return the keyspace for the request
*/
- public String keyspace()
+ public Name keyspace()
{
return keyspace;
}
diff --git a/src/main/dist/conf/sidecar.yaml b/src/main/dist/conf/sidecar.yaml
index d61a39c..884e183 100644
--- a/src/main/dist/conf/sidecar.yaml
+++ b/src/main/dist/conf/sidecar.yaml
@@ -135,6 +135,7 @@ cassandra_input_validation:
- system_auth
- system_views
- system_virtual_schema
- allowed_chars_for_directory: "[a-zA-Z0-9_-]+"
+ allowed_chars_for_directory: "[a-zA-Z][a-zA-Z0-9_]{0,47}"
+ allowed_chars_for_quoted_name: "[a-zA-Z_0-9]{1,48}"
allowed_chars_for_component_name:
"[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)"
allowed_chars_for_restricted_component_name: "[a-zA-Z0-9_-]+(.db|TOC.txt)"
diff --git
a/src/main/java/org/apache/cassandra/sidecar/config/CassandraInputValidationConfiguration.java
b/src/main/java/org/apache/cassandra/sidecar/config/CassandraInputValidationConfiguration.java
index f0daca8..98205a1 100644
---
a/src/main/java/org/apache/cassandra/sidecar/config/CassandraInputValidationConfiguration.java
+++
b/src/main/java/org/apache/cassandra/sidecar/config/CassandraInputValidationConfiguration.java
@@ -31,10 +31,18 @@ public interface CassandraInputValidationConfiguration
Set<String> forbiddenKeyspaces();
/**
- * @return a regular expression for an allowed pattern for directory names
- * (i.e. keyspace directory name or table directory name)
+ * @return a regular expression for an allowed pattern for name of
keyspaces and tables defined by the grammar in
+ * <a
href="https://cassandra.apache.org/doc/4.1/cassandra/cql/ddl.html#common-definitions">Cassandra
CQL common
+ * definitions</a>
*/
- String allowedPatternForDirectory();
+ String allowedPatternForName();
+
+ /**
+ * @return a regular expression for an allowed pattern for <i>quoted</i>
names of keyspace and tables defined by
+ * the grammar in <a
href="https://cassandra.apache.org/doc/4.1/cassandra/cql/ddl.html#common-definitions">
+ * Cassandra CQL common definitions</a>
+ */
+ String allowedPatternForQuotedName();
/**
* @return a regular expression for an allowed pattern for component names
diff --git
a/src/main/java/org/apache/cassandra/sidecar/config/yaml/CassandraInputValidationConfigurationImpl.java
b/src/main/java/org/apache/cassandra/sidecar/config/yaml/CassandraInputValidationConfigurationImpl.java
index 97d6351..bd5e8f8 100644
---
a/src/main/java/org/apache/cassandra/sidecar/config/yaml/CassandraInputValidationConfigurationImpl.java
+++
b/src/main/java/org/apache/cassandra/sidecar/config/yaml/CassandraInputValidationConfigurationImpl.java
@@ -40,8 +40,10 @@ public class CassandraInputValidationConfigurationImpl
implements CassandraInput
"system_auth",
"system_views",
"system_virtual_schema")));
- public static final String ALLOWED_CHARS_FOR_DIRECTORY_PROPERTY =
"allowed_chars_for_directory";
- public static final String DEFAULT_ALLOWED_CHARS_FOR_DIRECTORY =
"[a-zA-Z0-9_-]+";
+ public static final String ALLOWED_CHARS_FOR_NAME_PROPERTY =
"allowed_chars_for_directory";
+ public static final String DEFAULT_ALLOWED_CHARS_FOR_NAME =
"[a-zA-Z][a-zA-Z0-9_]{0,47}";
+ public static final String ALLOWED_CHARS_FOR_QUOTED_NAME_PROPERTY =
"allowed_chars_for_quoted_name";
+ public static final String DEFAULT_ALLOWED_CHARS_FOR_QUOTED_NAME =
"[a-zA-Z_0-9]{1,48}";
public static final String ALLOWED_CHARS_FOR_COMPONENT_NAME_PROPERTY =
"allowed_chars_for_component_name";
public static final String DEFAULT_ALLOWED_CHARS_FOR_COMPONENT_NAME =
"[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)";
@@ -52,8 +54,11 @@ public class CassandraInputValidationConfigurationImpl
implements CassandraInput
@JsonProperty(FORBIDDEN_KEYSPACES_PROPERTY)
protected final Set<String> forbiddenKeyspaces;
- @JsonProperty(value = ALLOWED_CHARS_FOR_DIRECTORY_PROPERTY, defaultValue =
DEFAULT_ALLOWED_CHARS_FOR_DIRECTORY)
- protected final String allowedPatternForDirectory;
+ @JsonProperty(value = ALLOWED_CHARS_FOR_NAME_PROPERTY, defaultValue =
DEFAULT_ALLOWED_CHARS_FOR_NAME)
+ protected final String allowedPatternForName;
+
+ @JsonProperty(value = ALLOWED_CHARS_FOR_QUOTED_NAME_PROPERTY, defaultValue
= DEFAULT_ALLOWED_CHARS_FOR_QUOTED_NAME)
+ protected final String allowedPatternForQuotedName;
@JsonProperty(value = ALLOWED_CHARS_FOR_COMPONENT_NAME_PROPERTY,
defaultValue = DEFAULT_ALLOWED_CHARS_FOR_COMPONENT_NAME)
@@ -66,24 +71,27 @@ public class CassandraInputValidationConfigurationImpl
implements CassandraInput
public CassandraInputValidationConfigurationImpl()
{
this(DEFAULT_FORBIDDEN_KEYSPACES,
- DEFAULT_ALLOWED_CHARS_FOR_DIRECTORY,
+ DEFAULT_ALLOWED_CHARS_FOR_NAME,
+ DEFAULT_ALLOWED_CHARS_FOR_QUOTED_NAME,
DEFAULT_ALLOWED_CHARS_FOR_COMPONENT_NAME,
DEFAULT_ALLOWED_CHARS_FOR_RESTRICTED_COMPONENT_NAME);
}
public CassandraInputValidationConfigurationImpl(Set<String>
forbiddenKeyspaces,
- String
allowedPatternForDirectory,
+ String
allowedPatternForName,
+ String
allowedPatternForQuotedName,
String
allowedPatternForComponentName,
String
allowedPatternForRestrictedComponentName)
{
this.forbiddenKeyspaces = forbiddenKeyspaces;
- this.allowedPatternForDirectory = allowedPatternForDirectory;
+ this.allowedPatternForName = allowedPatternForName;
+ this.allowedPatternForQuotedName = allowedPatternForQuotedName;
this.allowedPatternForComponentName = allowedPatternForComponentName;
this.allowedPatternForRestrictedComponentName =
allowedPatternForRestrictedComponentName;
}
/**
- * @return a set of forbidden keyspaces
+ * {@inheritDoc}
*/
@Override
@JsonProperty(FORBIDDEN_KEYSPACES_PROPERTY)
@@ -93,18 +101,27 @@ public class CassandraInputValidationConfigurationImpl
implements CassandraInput
}
/**
- * @return a regular expression for an allowed pattern for directory names
- * (i.e. keyspace directory name or table directory name)
+ * {@inheritDoc}
+ */
+ @Override
+ @JsonProperty(value = ALLOWED_CHARS_FOR_NAME_PROPERTY, defaultValue =
DEFAULT_ALLOWED_CHARS_FOR_NAME)
+ public String allowedPatternForName()
+ {
+ return allowedPatternForName;
+ }
+
+ /**
+ * {@inheritDoc}
*/
@Override
- @JsonProperty(value = ALLOWED_CHARS_FOR_DIRECTORY_PROPERTY, defaultValue =
DEFAULT_ALLOWED_CHARS_FOR_DIRECTORY)
- public String allowedPatternForDirectory()
+ @JsonProperty(value = ALLOWED_CHARS_FOR_QUOTED_NAME_PROPERTY, defaultValue
= DEFAULT_ALLOWED_CHARS_FOR_QUOTED_NAME)
+ public String allowedPatternForQuotedName()
{
- return allowedPatternForDirectory;
+ return allowedPatternForQuotedName;
}
/**
- * @return a regular expression for an allowed pattern for component names
+ * {@inheritDoc}
*/
@Override
@JsonProperty(value = ALLOWED_CHARS_FOR_COMPONENT_NAME_PROPERTY,
@@ -115,7 +132,7 @@ public class CassandraInputValidationConfigurationImpl
implements CassandraInput
}
/**
- * @return a regular expression to an allowed pattern for a subset of
component names
+ * {@inheritDoc}
*/
@Override
@JsonProperty(value = ALLOWED_CHARS_FOR_RESTRICTED_COMPONENT_NAME_PROPERTY,
diff --git a/src/main/java/org/apache/cassandra/sidecar/data/RingRequest.java
b/src/main/java/org/apache/cassandra/sidecar/data/RingRequest.java
index f04ae11..8dd7ec3 100644
--- a/src/main/java/org/apache/cassandra/sidecar/data/RingRequest.java
+++ b/src/main/java/org/apache/cassandra/sidecar/data/RingRequest.java
@@ -20,13 +20,15 @@ package org.apache.cassandra.sidecar.data;
import java.util.Objects;
+import org.apache.cassandra.sidecar.common.data.Name;
+
/**
* Holder class for the {@link org.apache.cassandra.sidecar.routes.RingHandler}
* request parameters
*/
public class RingRequest
{
- private final String keyspace;
+ private final Name keyspace;
/**
* Constructs a ring request
@@ -39,9 +41,9 @@ public class RingRequest
/**
* Constructs a ring request with an optional {@code keyspace} and a
{@code resolveIp} parameter.
*
- * @param keyspace the keyspace in Cassandra
+ * @param keyspace the keyspace in Cassandra
*/
- public RingRequest(String keyspace)
+ public RingRequest(Name keyspace)
{
this.keyspace = keyspace;
}
@@ -49,7 +51,7 @@ public class RingRequest
/**
* @return the keyspace for the request
*/
- public String keyspace()
+ public Name keyspace()
{
return keyspace;
}
@@ -82,7 +84,7 @@ public class RingRequest
public String toString()
{
return "RingRequest{" +
- "keyspace='" + keyspace +
+ "keyspace='" + (keyspace != null ? keyspace.name() : null) +
"'}";
}
}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/data/SSTableImportRequest.java
b/src/main/java/org/apache/cassandra/sidecar/data/SSTableImportRequest.java
index 3243943..d917f42 100644
--- a/src/main/java/org/apache/cassandra/sidecar/data/SSTableImportRequest.java
+++ b/src/main/java/org/apache/cassandra/sidecar/data/SSTableImportRequest.java
@@ -143,7 +143,7 @@ public class SSTableImportRequest extends SSTableUploads
&& copyData == that.copyData
&& uploadId().equals(that.uploadId())
&& keyspace().equals(that.keyspace())
- && tableName().equals(that.tableName());
+ && table().equals(that.table());
}
/**
@@ -151,7 +151,7 @@ public class SSTableImportRequest extends SSTableUploads
*/
public int hashCode()
{
- return Objects.hash(uploadId(), keyspace(), tableName(), resetLevel,
clearRepaired, verifySSTables,
+ return Objects.hash(uploadId(), keyspace(), table(), resetLevel,
clearRepaired, verifySSTables,
verifyTokens, invalidateCaches, extendedVerify,
copyData);
}
@@ -163,7 +163,7 @@ public class SSTableImportRequest extends SSTableUploads
return "SSTableUploadRequest{" +
"uploadId='" + uploadId() + '\'' +
", keyspace='" + keyspace() + '\'' +
- ", tableName='" + tableName() + '\'' +
+ ", tableName='" + table() + '\'' +
", resetLevel=" + resetLevel +
", clearRepaired=" + clearRepaired +
", verifySSTables=" + verifySSTables +
diff --git
a/src/main/java/org/apache/cassandra/sidecar/data/SSTableUploadRequest.java
b/src/main/java/org/apache/cassandra/sidecar/data/SSTableUploadRequest.java
index 16d352b..cef1491 100644
--- a/src/main/java/org/apache/cassandra/sidecar/data/SSTableUploadRequest.java
+++ b/src/main/java/org/apache/cassandra/sidecar/data/SSTableUploadRequest.java
@@ -73,7 +73,7 @@ public class SSTableUploadRequest extends SSTableUploads
return "SSTableUploadRequest{" +
"uploadId='" + uploadId() + '\'' +
", keyspace='" + keyspace() + '\'' +
- ", tableName='" + tableName() + '\'' +
+ ", tableName='" + table() + '\'' +
", component='" + component + '\'' +
", expectedChecksum='" + expectedChecksum + '\'' +
'}';
diff --git a/src/main/java/org/apache/cassandra/sidecar/data/SchemaRequest.java
b/src/main/java/org/apache/cassandra/sidecar/data/SchemaRequest.java
index c7955e0..6c4fb5d 100644
--- a/src/main/java/org/apache/cassandra/sidecar/data/SchemaRequest.java
+++ b/src/main/java/org/apache/cassandra/sidecar/data/SchemaRequest.java
@@ -17,6 +17,7 @@
*/
package org.apache.cassandra.sidecar.data;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.jetbrains.annotations.Nullable;
/**
@@ -25,7 +26,7 @@ import org.jetbrains.annotations.Nullable;
*/
public class SchemaRequest
{
- private final String keyspace;
+ private final Name keyspace;
/**
* Constructs a {@link SchemaRequest} with the {@link
org.jetbrains.annotations.Nullable} {@code keyspace}.
@@ -33,6 +34,16 @@ public class SchemaRequest
* @param keyspace the keyspace in Cassandra
*/
public SchemaRequest(@Nullable String keyspace)
+ {
+ this(keyspace == null ? null : new Name(keyspace));
+ }
+
+ /**
+ * Constructs a {@link SchemaRequest} with the {@link
org.jetbrains.annotations.Nullable} {@code keyspace}.
+ *
+ * @param keyspace the keyspace in Cassandra
+ */
+ public SchemaRequest(@Nullable Name keyspace)
{
this.keyspace = keyspace;
}
@@ -40,7 +51,7 @@ public class SchemaRequest
/**
* @return the keyspace in Cassandra
*/
- public String keyspace()
+ public Name keyspace()
{
return keyspace;
}
@@ -51,7 +62,7 @@ public class SchemaRequest
public String toString()
{
return "SchemaRequest{" +
- "keyspace='" + keyspace() + '\'' +
+ "keyspace='" + keyspace + '\'' +
'}';
}
}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
b/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
index e8d1304..c1568d3 100644
--- a/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
@@ -32,6 +32,7 @@ import io.vertx.ext.web.RoutingContext;
import io.vertx.ext.web.handler.HttpException;
import
org.apache.cassandra.sidecar.adapters.base.exception.OperationUnavailableException;
import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.QualifiedTableName;
import
org.apache.cassandra.sidecar.common.exceptions.JmxAuthenticationException;
import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
@@ -225,8 +226,7 @@ public abstract class AbstractHandler<T> implements
Handler<RoutingContext>
protected QualifiedTableName qualifiedTableName(RoutingContext context,
boolean required)
{
return new QualifiedTableName(keyspace(context, required),
- tableName(context, required),
- required);
+ tableName(context, required));
}
/**
@@ -236,7 +236,7 @@ public abstract class AbstractHandler<T> implements
Handler<RoutingContext>
* @param required whether the keyspace is required
* @return the validated keyspace name from the context
*/
- protected String keyspace(RoutingContext context, boolean required)
+ protected Name keyspace(RoutingContext context, boolean required)
{
String keyspace = context.pathParam(KEYSPACE_PATH_PARAM);
if (required || keyspace != null)
@@ -253,7 +253,7 @@ public abstract class AbstractHandler<T> implements
Handler<RoutingContext>
* @param required whether the table name is required
* @return the validated table name from the context
*/
- private String tableName(RoutingContext context, boolean required)
+ private Name tableName(RoutingContext context, boolean required)
{
String tableName = context.pathParam(TABLE_PATH_PARAM);
if (required || tableName != null)
diff --git
a/src/main/java/org/apache/cassandra/sidecar/routes/SchemaHandler.java
b/src/main/java/org/apache/cassandra/sidecar/routes/SchemaHandler.java
index 808003e..96e1f4b 100644
--- a/src/main/java/org/apache/cassandra/sidecar/routes/SchemaHandler.java
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/SchemaHandler.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
import org.apache.cassandra.sidecar.data.SchemaRequest;
import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+import org.apache.cassandra.sidecar.utils.MetadataUtils;
import static
org.apache.cassandra.sidecar.utils.HttpExceptions.cassandraServiceUnavailable;
import static
org.apache.cassandra.sidecar.utils.HttpExceptions.wrapHttpException;
@@ -96,7 +97,8 @@ public class SchemaHandler extends
AbstractHandler<SchemaRequest>
}
// retrieve keyspace metadata
- KeyspaceMetadata ksMetadata =
metadata.getKeyspace(requestParams.keyspace());
+ KeyspaceMetadata ksMetadata = MetadataUtils.keyspace(metadata,
requestParams.keyspace());
+
if (ksMetadata == null)
{
// set request as failed and return
@@ -107,7 +109,8 @@ public class SchemaHandler extends
AbstractHandler<SchemaRequest>
return;
}
- SchemaResponse schemaResponse = new
SchemaResponse(requestParams.keyspace(), ksMetadata.exportAsString());
+ SchemaResponse schemaResponse = new
SchemaResponse(requestParams.keyspace().name(),
+
ksMetadata.exportAsString());
context.json(schemaResponse);
}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/routes/SnapshotsHandler.java
b/src/main/java/org/apache/cassandra/sidecar/routes/SnapshotsHandler.java
index f6669e4..7adb465 100644
--- a/src/main/java/org/apache/cassandra/sidecar/routes/SnapshotsHandler.java
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/SnapshotsHandler.java
@@ -214,7 +214,7 @@ public class SnapshotsHandler extends
AbstractHandler<SnapshotRequest>
dataDirectoryIndex,
directory.snapshotName,
directory.keyspace,
- directory.tableName,
+
maybeRemoveTableId(directory.tableName),
snapshotFile.path.substring(fileNameIndex)));
}
return response;
@@ -354,4 +354,20 @@ public class SnapshotsHandler extends
AbstractHandler<SnapshotRequest>
{
validator.validateSnapshotName(request.snapshotName());
}
+
+ /**
+ * Removes the table UUID portion from the table name if present.
+ *
+ * @param tableName the table name with or without the UUID
+ * @return the table name without the UUID
+ */
+ private String maybeRemoveTableId(String tableName)
+ {
+ int dashIndex = tableName.lastIndexOf("-");
+ if (dashIndex > 0)
+ {
+ return tableName.substring(0, dashIndex);
+ }
+ return tableName;
+ }
}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandler.java
b/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandler.java
index ef8b366..97b8290 100644
---
a/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandler.java
+++
b/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandler.java
@@ -115,8 +115,8 @@ public class SSTableImportHandler extends
AbstractHandler<SSTableImportRequest>
{
context.json(new SSTableImportResponse(true,
request.uploadId(),
-
request.keyspace(),
-
request.tableName()));
+
request.keyspace().name(),
+
request.table().name()));
logger.debug("ImportHandler completed
request={}, remoteAddress={}, instance={}",
request, remoteAddress, host);
}
@@ -183,8 +183,8 @@ public class SSTableImportHandler extends
AbstractHandler<SSTableImportRequest>
{
return new SSTableImporter.ImportOptions.Builder()
.host(host)
- .keyspace(request.keyspace())
- .tableName(request.tableName())
+ .keyspace(request.keyspace().name())
+ .tableName(request.table().name())
.directory(uploadDirectory)
.uploadId(request.uploadId())
.resetLevel(request.resetLevel())
diff --git
a/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandler.java
b/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandler.java
index 845dd4c..dc23141 100644
---
a/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandler.java
+++
b/src/main/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandler.java
@@ -43,6 +43,7 @@ import org.apache.cassandra.sidecar.stats.SSTableStats;
import org.apache.cassandra.sidecar.stats.SidecarStats;
import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+import org.apache.cassandra.sidecar.utils.MetadataUtils;
import org.apache.cassandra.sidecar.utils.SSTableUploader;
import org.apache.cassandra.sidecar.utils.SSTableUploadsPathBuilder;
@@ -187,7 +188,7 @@ public class SSTableUploadHandler extends
AbstractHandler<SSTableUploadRequest>
}
})
.compose(metadata -> {
- KeyspaceMetadata keyspaceMetadata =
metadata.getKeyspace(request.keyspace());
+ KeyspaceMetadata keyspaceMetadata =
MetadataUtils.keyspace(metadata, request.keyspace());
if (keyspaceMetadata == null)
{
String message = String.format("Invalid keyspace
'%s' supplied", request.keyspace());
@@ -195,10 +196,10 @@ public class SSTableUploadHandler extends
AbstractHandler<SSTableUploadRequest>
return
Future.failedFuture(wrapHttpException(HttpResponseStatus.BAD_REQUEST, message));
}
- if (keyspaceMetadata.getTable(request.tableName()) ==
null)
+ if (MetadataUtils.table(keyspaceMetadata,
request.table()) == null)
{
String message = String.format("Invalid table name
'%s' supplied for keyspace '%s'",
- request.tableName(),
request.keyspace());
+ request.table(),
request.keyspace());
logger.error(message);
return
Future.failedFuture(wrapHttpException(HttpResponseStatus.BAD_REQUEST, message));
}
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 655b840..7974b35 100644
---
a/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotPathBuilder.java
+++
b/src/main/java/org/apache/cassandra/sidecar/snapshots/SnapshotPathBuilder.java
@@ -289,7 +289,8 @@ public class SnapshotPathBuilder extends BaseFileSystem
{
Preconditions.checkArgument(!secondaryIndexName.isEmpty(),
"secondaryIndexName cannot be empty");
Preconditions.checkArgument(secondaryIndexName.charAt(0) == '.',
"Invalid secondary index name");
- validator.validatePattern(secondaryIndexName.substring(1),
"secondary index");
+ String indexName = secondaryIndexName.substring(1);
+ validator.validatePattern(indexName, indexName, "secondary index",
false);
}
validator.validateRestrictedComponentName(request.componentName());
}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/utils/CassandraInputValidator.java
b/src/main/java/org/apache/cassandra/sidecar/utils/CassandraInputValidator.java
index 838388e..086b341 100644
---
a/src/main/java/org/apache/cassandra/sidecar/utils/CassandraInputValidator.java
+++
b/src/main/java/org/apache/cassandra/sidecar/utils/CassandraInputValidator.java
@@ -25,6 +25,7 @@ import com.google.inject.Inject;
import com.google.inject.Singleton;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.vertx.ext.web.handler.HttpException;
+import org.apache.cassandra.sidecar.common.data.Name;
import
org.apache.cassandra.sidecar.config.CassandraInputValidationConfiguration;
import
org.apache.cassandra.sidecar.config.yaml.CassandraInputValidationConfigurationImpl;
import org.jetbrains.annotations.NotNull;
@@ -65,13 +66,16 @@ public class CassandraInputValidator
* @throws HttpException when the {@code keyspace} contains invalid
characters in the name or when the
* keyspace is forbidden
*/
- public String validateKeyspaceName(@NotNull String keyspace)
+ public Name validateKeyspaceName(@NotNull String keyspace)
{
Objects.requireNonNull(keyspace, "keyspace must not be null");
- validatePattern(keyspace, "keyspace");
- if (validationConfiguration.forbiddenKeyspaces().contains(keyspace))
+ String unquoted = removeQuotesIfNecessary(keyspace);
+ // whether the input is quoted from the source
+ boolean isQuoted = !unquoted.equals(keyspace);
+ validatePattern(unquoted, keyspace, "keyspace", isQuoted);
+ if (validationConfiguration.forbiddenKeyspaces().contains(unquoted))
throw new HttpException(HttpResponseStatus.FORBIDDEN.code(),
"Forbidden keyspace: " + keyspace);
- return keyspace;
+ return new Name(unquoted, keyspace);
}
/**
@@ -83,11 +87,13 @@ public class CassandraInputValidator
* @throws NullPointerException when the {@code tableName} is {@code null}
* @throws HttpException when the {@code tableName} contains
invalid characters in the name
*/
- public String validateTableName(@NotNull String tableName)
+ public Name validateTableName(@NotNull String tableName)
{
Objects.requireNonNull(tableName, "tableName must not be null");
- validatePattern(tableName, "table name");
- return tableName;
+ String unquoted = removeQuotesIfNecessary(tableName);
+ boolean isQuoted = !unquoted.equals(tableName);
+ validatePattern(unquoted, tableName, "table name", isQuoted);
+ return new Name(unquoted, tableName);
}
/**
@@ -97,7 +103,7 @@ public class CassandraInputValidator
* @param snapshotName the name of the Cassandra snapshot to validate
* @return the validated {@code snapshotName}
* @throws NullPointerException when the {@code snapshotName} is {@code
null}
- * @throws HttpException when the {@code snapshotName} contains
inalid characters in the name
+ * @throws HttpException when the {@code snapshotName} contains
invalid characters in the name
*/
public String validateSnapshotName(@NotNull String snapshotName)
{
@@ -160,16 +166,39 @@ public class CassandraInputValidator
}
/**
- * Validates that the {@code input} matches the {@code patternWordChars}
+ * Validates that the {@code unquotedInput} matches the {@code
patternWordChars}
*
- * @param input the input
- * @param name a name for the exception
- * @throws HttpException when the {@code input} does not match the pattern
+ * @param unquotedInput the unquoted input
+ * @param quotedInput the original input used for the exception
message
+ * @param name a name for the exception message
+ * @param isQuotedFromSource whether the name was quoted from source
+ * @throws HttpException when the {@code unquotedInput} does not match the
pattern
*/
- public void validatePattern(String input, String name)
+ public void validatePattern(String unquotedInput, String quotedInput,
String name, boolean isQuotedFromSource)
{
- if
(!input.matches(validationConfiguration.allowedPatternForDirectory()))
+ String pattern = isQuotedFromSource
+ ?
validationConfiguration.allowedPatternForQuotedName()
+ : validationConfiguration.allowedPatternForName();
+
+ if (!unquotedInput.matches(pattern))
throw new HttpException(HttpResponseStatus.BAD_REQUEST.code(),
- "Invalid characters in " + name + ": " +
input);
+ "Invalid characters in " + name + ": " +
quotedInput);
+ }
+
+ /**
+ * Removes the surrounding quotes for the name, if the quotes are present.
Otherwise, returns the original
+ * input.
+ *
+ * @param name the name
+ * @return the {@code name} without surrounding quotes
+ */
+ static String removeQuotesIfNecessary(String name)
+ {
+ if (name == null || name.length() <= 1
+ || name.charAt(0) != '"' || name.charAt(name.length() - 1) != '"')
+ {
+ return name;
+ }
+ return name.substring(1, name.length() - 1);
}
}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/utils/MetadataUtils.java
b/src/main/java/org/apache/cassandra/sidecar/utils/MetadataUtils.java
new file mode 100644
index 0000000..c902b4b
--- /dev/null
+++ b/src/main/java/org/apache/cassandra/sidecar/utils/MetadataUtils.java
@@ -0,0 +1,57 @@
+/*
+ * 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.utils;
+
+import com.datastax.driver.core.KeyspaceMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.QueryOptions;
+import com.datastax.driver.core.TableMetadata;
+import org.apache.cassandra.sidecar.common.data.Name;
+
+/**
+ * Utilities for {@link Metadata} operations
+ */
+public class MetadataUtils
+{
+ /**
+ * Returns the metadata of a keyspace given its name.
+ *
+ * @param metadata the metadata object.
+ * @param keyspace the name of the keyspace for which metadata should be
returned.
+ * @return the metadata of the requested keyspace or {@code null} if
{@code keyspace} is not a
+ * known keyspace. Note that the result might be stale or null if metadata
was explicitly
+ * disabled with {@link QueryOptions#setMetadataEnabled(boolean)}.
+ */
+ public static KeyspaceMetadata keyspace(Metadata metadata, Name keyspace)
+ {
+ return metadata.getKeyspace(keyspace.maybeQuotedName());
+ }
+
+ /**
+ * Returns the metadata for a table contained in this keyspace.
+ *
+ * @param metadata the metadata object.
+ * @param table the name of table to retrieve
+ * @return the metadata for table {@code name} if it exists in this
keyspace, {@code null}
+ * otherwise.
+ */
+ public static TableMetadata table(KeyspaceMetadata metadata, Name table)
+ {
+ return metadata.getTable(table.maybeQuotedName());
+ }
+}
diff --git
a/src/main/java/org/apache/cassandra/sidecar/utils/SSTableUploadsPathBuilder.java
b/src/main/java/org/apache/cassandra/sidecar/utils/SSTableUploadsPathBuilder.java
index 07df6e6..42a20d5 100644
---
a/src/main/java/org/apache/cassandra/sidecar/utils/SSTableUploadsPathBuilder.java
+++
b/src/main/java/org/apache/cassandra/sidecar/utils/SSTableUploadsPathBuilder.java
@@ -75,7 +75,9 @@ public class SSTableUploadsPathBuilder extends BaseFileSystem
return validate(request)
.compose(validRequest -> resolveUploadIdDirectory(host,
request.uploadId()))
.compose(stagingDirectory ->
- resolveUploadDirectory(stagingDirectory,
request.keyspace(), request.tableName()));
+ resolveUploadDirectory(stagingDirectory,
+ request.keyspace().name(),
+ request.table().name()));
}
/**
@@ -152,8 +154,8 @@ public class SSTableUploadsPathBuilder extends
BaseFileSystem
.compose(validUploadId -> {
try
{
- validator.validateKeyspaceName(request.keyspace());
- validator.validateTableName(request.tableName());
+
validator.validateKeyspaceName(request.keyspace().name());
+ validator.validateTableName(request.table().name());
if (request instanceof SSTableUploadRequest)
{
diff --git
a/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
b/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
index 70bd110..01459d3 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
@@ -38,6 +38,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.datastax.driver.core.Metadata;
import com.datastax.driver.core.Session;
import com.google.inject.Guice;
import com.google.inject.Injector;
@@ -51,6 +52,7 @@ import io.vertx.junit5.VertxTestContext;
import org.apache.cassandra.sidecar.cluster.CassandraAdapterDelegate;
import org.apache.cassandra.sidecar.cluster.InstancesConfig;
import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.common.data.Name;
import org.apache.cassandra.sidecar.common.data.QualifiedTableName;
import org.apache.cassandra.sidecar.common.dns.DnsResolver;
import org.apache.cassandra.sidecar.server.MainModule;
@@ -71,6 +73,7 @@ public abstract class IntegrationTestBase
protected Logger logger = LoggerFactory.getLogger(this.getClass());
protected Vertx vertx;
protected Server server;
+ protected WebClient client;
protected static final String TEST_KEYSPACE = "testkeyspace";
private static final String TEST_TABLE_PREFIX = "testtable";
@@ -101,6 +104,7 @@ public abstract class IntegrationTestBase
});
}
+ client = WebClient.create(vertx);
server.start()
.onSuccess(s -> {
sidecarTestContext.registerInstanceConfigListener(this::healthCheck);
@@ -118,6 +122,7 @@ public abstract class IntegrationTestBase
void tearDown() throws InterruptedException
{
CountDownLatch closeLatch = new CountDownLatch(1);
+ client.close();
server.close().onSuccess(res -> closeLatch.countDown());
if (closeLatch.await(60, TimeUnit.SECONDS))
logger.info("Close event received before timeout.");
@@ -128,7 +133,6 @@ public abstract class IntegrationTestBase
protected void testWithClient(VertxTestContext context,
Consumer<WebClient> tester) throws Exception
{
- WebClient client = WebClient.create(vertx);
CassandraAdapterDelegate delegate =
sidecarTestContext.instancesConfig()
.instanceFromId(1)
.delegate();
@@ -170,9 +174,14 @@ public abstract class IntegrationTestBase
}
protected QualifiedTableName createTestTable(String createTableStatement)
+ {
+ return createTestTable(TEST_TABLE_PREFIX, createTableStatement);
+ }
+
+ protected QualifiedTableName createTestTable(String tablePrefix, String
createTableStatement)
{
Session session = maybeGetSession();
- QualifiedTableName tableName = uniqueTestTableFullName();
+ QualifiedTableName tableName = uniqueTestTableFullName(tablePrefix);
session.execute(String.format(createTableStatement, tableName));
return tableName;
}
@@ -184,9 +193,11 @@ public abstract class IntegrationTestBase
return session;
}
- private static QualifiedTableName uniqueTestTableFullName()
+ private static QualifiedTableName uniqueTestTableFullName(String
tablePrefix)
{
- return new QualifiedTableName(TEST_KEYSPACE, TEST_TABLE_PREFIX +
TEST_TABLE_ID.getAndIncrement());
+ String unquotedTableName = tablePrefix +
TEST_TABLE_ID.getAndIncrement();
+ return new QualifiedTableName(new Name(TEST_KEYSPACE,
Metadata.quoteIfNecessary(TEST_KEYSPACE)),
+ new Name(unquotedTableName,
Metadata.quoteIfNecessary(unquotedTableName)));
}
public List<Path> findChildFile(CassandraSidecarTestContext context,
String hostname, String target)
diff --git
a/src/test/integration/org/apache/cassandra/sidecar/routes/SchemaHandlerIntegrationTest.java
b/src/test/integration/org/apache/cassandra/sidecar/routes/SchemaHandlerIntegrationTest.java
new file mode 100644
index 0000000..32fabf9
--- /dev/null
+++
b/src/test/integration/org/apache/cassandra/sidecar/routes/SchemaHandlerIntegrationTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.junit.jupiter.api.extension.ExtendWith;
+
+import com.datastax.driver.core.Session;
+import io.vertx.ext.web.client.predicate.ResponsePredicate;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.common.data.SchemaResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration tests for the {@link SchemaHandler}
+ */
+@ExtendWith(VertxExtension.class)
+class SchemaHandlerIntegrationTest extends IntegrationTestBase
+{
+ @CassandraIntegrationTest
+ void schemaHandlerNoKeyspace(VertxTestContext context) throws Exception
+ {
+ String testRoute = "/api/v1/schema/keyspaces";
+ testWithClient(context, client -> {
+ client.get(server.actualPort(), "127.0.0.1", testRoute)
+ .expect(ResponsePredicate.SC_OK)
+ .send(context.succeeding(response -> {
+ SchemaResponse schemaResponse =
response.bodyAsJson(SchemaResponse.class);
+ assertThat(schemaResponse).isNotNull();
+ assertThat(schemaResponse.keyspace()).isNull();
+ assertThat(schemaResponse.schema()).isNotNull();
+ context.completeNow();
+ }));
+ });
+ }
+
+ @CassandraIntegrationTest
+ void schemaHandlerKeyspaceDoesNotExist(VertxTestContext context) throws
Exception
+ {
+ String testRoute = "/api/v1/schema/keyspaces/non_existent";
+ testWithClient(context, client -> {
+ client.get(server.actualPort(), "127.0.0.1", testRoute)
+ .expect(ResponsePredicate.SC_NOT_FOUND)
+ .send(context.succeedingThenComplete());
+ });
+ }
+
+ @CassandraIntegrationTest
+ void schemaHandlerWithKeyspace(VertxTestContext context) throws Exception
+ {
+ createTestKeyspace();
+
+ String testRoute = "/api/v1/schema/keyspaces/testkeyspace";
+ testWithClient(context, client -> {
+ client.get(server.actualPort(), "127.0.0.1", testRoute)
+ .expect(ResponsePredicate.SC_OK)
+ .send(context.succeeding(response -> {
+ SchemaResponse schemaResponse =
response.bodyAsJson(SchemaResponse.class);
+ assertThat(schemaResponse).isNotNull();
+
assertThat(schemaResponse.keyspace()).isEqualTo("testkeyspace");
+ assertThat(schemaResponse.schema()).isNotNull();
+ context.completeNow();
+ }));
+ });
+ }
+
+ @CassandraIntegrationTest
+ void schemaHandlerWithCaseSensitiveKeyspace(VertxTestContext context)
throws Exception
+ {
+ try (Session session = maybeGetSession())
+ {
+ session.execute("CREATE KEYSPACE \"Cycling\"" +
+ " WITH REPLICATION = { 'class' :
'NetworkTopologyStrategy', 'replication_factor' : 1 };");
+ }
+
+ String testRoute = "/api/v1/schema/keyspaces/\"Cycling\"";
+ testWithClient(context, client -> {
+ client.get(server.actualPort(), "127.0.0.1", testRoute)
+ .expect(ResponsePredicate.SC_OK)
+ .send(context.succeeding(response -> {
+ SchemaResponse schemaResponse =
response.bodyAsJson(SchemaResponse.class);
+ assertThat(schemaResponse).isNotNull();
+
assertThat(schemaResponse.keyspace()).isEqualTo("Cycling");
+ assertThat(schemaResponse.schema()).isNotNull();
+ context.completeNow();
+ }));
+ });
+ }
+
+ @CassandraIntegrationTest
+ void schemaHandlerWithReservedKeywordKeyspace(VertxTestContext context)
throws Exception
+ {
+ try (Session session = maybeGetSession())
+ {
+ session.execute("CREATE KEYSPACE \"keyspace\"" +
+ " WITH REPLICATION = { 'class' :
'NetworkTopologyStrategy', 'replication_factor' : 1 };");
+ }
+
+ String testRoute = "/api/v1/schema/keyspaces/\"keyspace\"";
+ testWithClient(context, client -> {
+ client.get(server.actualPort(), "127.0.0.1", testRoute)
+ .expect(ResponsePredicate.SC_OK)
+ .send(context.succeeding(response -> {
+ SchemaResponse schemaResponse =
response.bodyAsJson(SchemaResponse.class);
+ assertThat(schemaResponse).isNotNull();
+
assertThat(schemaResponse.keyspace()).isEqualTo("keyspace");
+ assertThat(schemaResponse.schema()).isNotNull();
+ context.completeNow();
+ }));
+ });
+ }
+}
diff --git
a/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
b/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
index 424c179..2e2c797 100644
---
a/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
+++
b/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
@@ -32,7 +32,6 @@ import io.vertx.junit5.VertxExtension;
import io.vertx.junit5.VertxTestContext;
import org.apache.cassandra.sidecar.IntegrationTestBase;
import org.apache.cassandra.sidecar.common.data.QualifiedTableName;
-import org.apache.cassandra.sidecar.test.CassandraSidecarTestContext;
import org.apache.cassandra.testing.CassandraIntegrationTest;
import static io.netty.handler.codec.http.HttpResponseStatus.OK;
@@ -45,7 +44,7 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
void createSnapshotEndpointFailsWhenKeyspaceDoesNotExist(VertxTestContext
context) throws InterruptedException
{
WebClient client = WebClient.create(vertx);
- String testRoute =
"/api/v1/keyspaces/non-existent/tables/testtable/snapshots/my-snapshot";
+ String testRoute =
"/api/v1/keyspaces/non_existent/tables/testtable/snapshots/my-snapshot";
client.put(server.actualPort(), "127.0.0.1", testRoute)
.expect(ResponsePredicate.SC_NOT_FOUND)
.send(context.succeedingThenComplete());
@@ -60,7 +59,7 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
createTestKeyspace();
WebClient client = WebClient.create(vertx);
- String testRoute =
"/api/v1/keyspaces/testkeyspace/tables/non-existent/snapshots/my-snapshot";
+ String testRoute =
"/api/v1/keyspaces/testkeyspace/tables/non_existent/snapshots/my-snapshot";
client.put(server.actualPort(), "127.0.0.1", testRoute)
.expect(ResponsePredicate.SC_NOT_FOUND)
.send(context.succeedingThenComplete());
@@ -73,11 +72,11 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
throws InterruptedException
{
createTestKeyspace();
- String table = createTestTableAndPopulate(sidecarTestContext);
+ QualifiedTableName tableName = createTestTableAndPopulate();
WebClient client = WebClient.create(vertx);
String testRoute =
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/my-snapshot",
- TEST_KEYSPACE, table);
+ tableName.maybeQuotedKeyspace(),
tableName.maybeQuotedTableName());
client.put(server.actualPort(), "127.0.0.1", testRoute)
.expect(ResponsePredicate.SC_OK)
.send(context.succeeding(response ->
@@ -99,11 +98,40 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
throws InterruptedException
{
createTestKeyspace();
- String table = createTestTableAndPopulate(sidecarTestContext);
+ QualifiedTableName tableName = createTestTableAndPopulate();
WebClient client = WebClient.create(vertx);
String testRoute =
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/my-snapshot",
- TEST_KEYSPACE, table);
+ tableName.maybeQuotedKeyspace(),
tableName.maybeQuotedTableName());
+ client.put(server.actualPort(), "127.0.0.1", testRoute)
+ .expect(ResponsePredicate.SC_OK)
+ .send(context.succeeding(response -> context.verify(() -> {
+ assertThat(response.statusCode()).isEqualTo(OK.code());
+
+ // validate that the snapshot is created
+ List<Path> found = findChildFile(sidecarTestContext,
"127.0.0.1",
+ "my-snapshot");
+ assertThat(found).isNotEmpty()
+ .anyMatch(p ->
p.toString().endsWith("manifest.json"))
+ .anyMatch(p ->
p.toString().endsWith("schema.cql"))
+ .anyMatch(p ->
p.toString().endsWith("-big-Data.db"));
+
+ context.completeNow();
+ })));
+ // wait until test completes
+ assertThat(context.awaitCompletion(30, TimeUnit.SECONDS)).isTrue();
+ }
+
+ @CassandraIntegrationTest
+ void testCreateSnapshotEndpointWithMixedCaseTableName(VertxTestContext
context)
+ throws InterruptedException
+ {
+ createTestKeyspace();
+ QualifiedTableName tableName =
createTestTableAndPopulate("QuOtEdTaBlENaMe");
+
+ WebClient client = WebClient.create(vertx);
+ String testRoute =
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/my-snapshot",
+ tableName.maybeQuotedKeyspace(),
tableName.maybeQuotedTableName());
client.put(server.actualPort(), "127.0.0.1", testRoute)
.expect(ResponsePredicate.SC_OK)
.send(context.succeeding(response -> context.verify(() -> {
@@ -126,7 +154,7 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
@CassandraIntegrationTest
void deleteSnapshotFailsWhenKeyspaceDoesNotExist(VertxTestContext context)
throws InterruptedException
{
- String testRoute =
"/api/v1/keyspaces/non-existent/tables/testtable/snapshots/my-snapshot";
+ String testRoute =
"/api/v1/keyspaces/non_existent/tables/testtable/snapshots/my-snapshot";
assertNotFoundOnDeleteSnapshot(context, testRoute);
}
@@ -135,9 +163,9 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
throws InterruptedException
{
createTestKeyspace();
- createTestTableAndPopulate(sidecarTestContext);
+ createTestTableAndPopulate();
- String testRoute =
"/api/v1/keyspaces/testkeyspace/tables/non-existent/snapshots/my-snapshot";
+ String testRoute =
"/api/v1/keyspaces/testkeyspace/tables/non_existent/snapshots/my-snapshot";
assertNotFoundOnDeleteSnapshot(context, testRoute);
}
@@ -146,10 +174,10 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
throws InterruptedException
{
createTestKeyspace();
- String table = createTestTableAndPopulate(sidecarTestContext);
+ QualifiedTableName tableName = createTestTableAndPopulate();
- String testRoute =
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/non-existent",
- TEST_KEYSPACE, table);
+ String testRoute =
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/non_existent",
+ tableName.maybeQuotedKeyspace(),
tableName.maybeQuotedTableName());
assertNotFoundOnDeleteSnapshot(context, testRoute);
}
@@ -159,12 +187,13 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
throws InterruptedException
{
createTestKeyspace();
- String table = createTestTableAndPopulate(sidecarTestContext);
+ QualifiedTableName tableName = createTestTableAndPopulate();
WebClient client = WebClient.create(vertx);
String snapshotName = "my-snapshot" + UUID.randomUUID();
String testRoute =
String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/%s",
- TEST_KEYSPACE, table, snapshotName);
+ tableName.maybeQuotedKeyspace(),
tableName.maybeQuotedTableName(),
+ snapshotName);
// first create the snapshot
client.put(server.actualPort(), "127.0.0.1", testRoute)
@@ -199,7 +228,19 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
assertThat(context.awaitCompletion(30, TimeUnit.SECONDS)).isTrue();
}
- private String createTestTableAndPopulate(CassandraSidecarTestContext
cassandraTestContext)
+ private QualifiedTableName createTestTableAndPopulate(String
tableNamePrefix)
+ {
+ QualifiedTableName tableName = createTestTable(tableNamePrefix,
+ "CREATE TABLE %s (id text PRIMARY KEY, name text);");
+ 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 QualifiedTableName createTestTableAndPopulate()
{
QualifiedTableName tableName = createTestTable(
"CREATE TABLE %s (id text PRIMARY KEY, name text);");
@@ -208,7 +249,7 @@ class SnapshotsHandlerIntegrationTest extends
IntegrationTestBase
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.tableName();
+ return tableName;
}
private void assertNotFoundOnDeleteSnapshot(VertxTestContext context,
String testRoute) throws InterruptedException
diff --git a/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
b/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
index 8077214..50d0681 100644
--- a/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
@@ -83,10 +83,8 @@ public class ThrottleTest
@Test
void testStreamRequestsThrottled() throws Exception
{
- String testRoute = "/keyspaces/TestKeyspace" +
-
"/tables/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b" +
- "/snapshots/TestSnapshot" +
-
"/components/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db";
+ String testRoute =
"/keyspaces/TestKeyspace/tables/TestTable/snapshots/TestSnapshot" +
+ "/components/nb-1-big-Data.db";
for (int i = 0; i < 20; i++)
{
diff --git
a/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
b/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
index 36677df..b0378a7 100644
---
a/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
+++
b/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
@@ -86,7 +86,8 @@ class SidecarConfigurationTest
configuration.cassandraInputValidationConfiguration();
assertThat(validationConfiguration.forbiddenKeyspaces()).contains("a",
"b", "c");
-
assertThat(validationConfiguration.allowedPatternForDirectory()).isEqualTo("[a-z]+");
+
assertThat(validationConfiguration.allowedPatternForName()).isEqualTo("[a-z]+");
+
assertThat(validationConfiguration.allowedPatternForQuotedName()).isEqualTo("[A-Z]+");
assertThat(validationConfiguration.allowedPatternForComponentName())
.isEqualTo("(.db|.cql|.json|.crc32|TOC.txt)");
assertThat(validationConfiguration.allowedPatternForRestrictedComponentName())
@@ -315,7 +316,8 @@ class SidecarConfigurationTest
"system_auth",
"system_views",
"system_virtual_schema");
-
assertThat(config.allowedPatternForDirectory()).isEqualTo("[a-zA-Z0-9_-]+");
+
assertThat(config.allowedPatternForName()).isEqualTo("[a-zA-Z][a-zA-Z0-9_]{0,47}");
+
assertThat(config.allowedPatternForQuotedName()).isEqualTo("[a-zA-Z_0-9]{1,48}");
assertThat(config.allowedPatternForComponentName())
.isEqualTo("[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)");
assertThat(config.allowedPatternForRestrictedComponentName()).isEqualTo("[a-zA-Z0-9_-]+(.db|TOC.txt)");
diff --git
a/src/test/java/org/apache/cassandra/sidecar/data/RingRequestTest.java
b/src/test/java/org/apache/cassandra/sidecar/data/RingRequestTest.java
index 90a140a..4fca39d 100644
--- a/src/test/java/org/apache/cassandra/sidecar/data/RingRequestTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/data/RingRequestTest.java
@@ -20,6 +20,8 @@ package org.apache.cassandra.sidecar.data;
import org.junit.jupiter.api.Test;
+import org.apache.cassandra.sidecar.common.data.Name;
+
import static org.assertj.core.api.Assertions.assertThat;
/**
@@ -38,16 +40,17 @@ class RingRequestTest
@Test
void testConstructorWithParams()
{
- RingRequest request = new RingRequest("valid_keyspace");
+ RingRequest request = new RingRequest(new Name("valid_keyspace"));
assertThat(request).isNotNull();
- assertThat(request.keyspace()).isEqualTo("valid_keyspace");
+ assertThat(request.keyspace()).isNotNull();
+ assertThat(request.keyspace().name()).isEqualTo("valid_keyspace");
}
@Test
void testToString()
{
- RingRequest request1 = new RingRequest("valid_keyspace");
- RingRequest request2 = new RingRequest("ks2");
+ RingRequest request1 = new RingRequest(new Name("valid_keyspace"));
+ RingRequest request2 = new RingRequest(new Name("ks2"));
assertThat(request1).hasToString("RingRequest{keyspace='valid_keyspace'}");
assertThat(request2).hasToString("RingRequest{keyspace='ks2'}");
}
@@ -55,9 +58,9 @@ class RingRequestTest
@Test
void testEquals()
{
- RingRequest request1 = new RingRequest("ks");
- RingRequest request2 = new RingRequest("ks");
- RingRequest request3 = new RingRequest("ks5");
+ RingRequest request1 = new RingRequest(new Name("ks"));
+ RingRequest request2 = new RingRequest(new Name("ks"));
+ RingRequest request3 = new RingRequest(new Name("ks5"));
assertThat(request1).isEqualTo(request2);
assertThat(request1).isNotSameAs(request2);
assertThat(request1).isNotEqualTo(request3);
@@ -66,9 +69,9 @@ class RingRequestTest
@Test
void testHashCode()
{
- RingRequest request1 = new RingRequest("ks");
- RingRequest request2 = new RingRequest("ks");
- RingRequest request3 = new RingRequest("ks5");
+ RingRequest request1 = new RingRequest(new Name("ks"));
+ RingRequest request2 = new RingRequest(new Name("ks"));
+ RingRequest request3 = new RingRequest(new Name("ks5"));
assertThat(request1).hasSameHashCodeAs(request2);
assertThat(request1).doesNotHaveSameHashCodeAs(request3);
}
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V30.java
b/src/test/java/org/apache/cassandra/sidecar/mocks/V30.java
similarity index 100%
rename from common/src/main/java/org/apache/cassandra/sidecar/mocks/V30.java
rename to src/test/java/org/apache/cassandra/sidecar/mocks/V30.java
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V40.java
b/src/test/java/org/apache/cassandra/sidecar/mocks/V40.java
similarity index 100%
rename from common/src/main/java/org/apache/cassandra/sidecar/mocks/V40.java
rename to src/test/java/org/apache/cassandra/sidecar/mocks/V40.java
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V41.java
b/src/test/java/org/apache/cassandra/sidecar/mocks/V41.java
similarity index 100%
rename from common/src/main/java/org/apache/cassandra/sidecar/mocks/V41.java
rename to src/test/java/org/apache/cassandra/sidecar/mocks/V41.java
diff --git
a/src/test/java/org/apache/cassandra/sidecar/routes/SnapshotsHandlerTest.java
b/src/test/java/org/apache/cassandra/sidecar/routes/SnapshotsHandlerTest.java
index 2f28e31..83d64fd 100644
---
a/src/test/java/org/apache/cassandra/sidecar/routes/SnapshotsHandlerTest.java
+++
b/src/test/java/org/apache/cassandra/sidecar/routes/SnapshotsHandlerTest.java
@@ -106,7 +106,7 @@ public class SnapshotsHandlerTest
public void testRouteSucceedsWithKeyspaceAndTableName(VertxTestContext
context)
{
WebClient client = WebClient.create(vertx);
- String testRoute =
"/api/v1/keyspaces/keyspace1/tables/table1-1234/snapshots/snapshot1";
+ String testRoute =
"/api/v1/keyspaces/keyspace1/tables/table1/snapshots/snapshot1";
ListSnapshotFilesResponse.FileInfo fileInfoExpected =
new ListSnapshotFilesResponse.FileInfo(11,
"localhost",
@@ -114,7 +114,7 @@ public class SnapshotsHandlerTest
0,
"snapshot1",
"keyspace1",
- "table1-1234",
+ "table1",
"1.db");
ListSnapshotFilesResponse.FileInfo fileInfoNotExpected =
new ListSnapshotFilesResponse.FileInfo(11,
@@ -123,7 +123,7 @@ public class SnapshotsHandlerTest
0,
"snapshot1",
"keyspace1",
- "table1-1234",
+ "table1",
"2.db");
client.get(server.actualPort(), "localhost", testRoute)
@@ -141,7 +141,7 @@ public class SnapshotsHandlerTest
public void testRouteSucceedsIncludeSecondaryIndexes(VertxTestContext
context)
{
WebClient client = WebClient.create(vertx);
- String testRoute = "/api/v1/keyspaces/keyspace1/tables/table1-1234" +
+ String testRoute = "/api/v1/keyspaces/keyspace1/tables/table1" +
"/snapshots/snapshot1?includeSecondaryIndexFiles=true";
List<ListSnapshotFilesResponse.FileInfo> fileInfoExpected =
Arrays.asList(
new ListSnapshotFilesResponse.FileInfo(11,
@@ -150,7 +150,7 @@ public class SnapshotsHandlerTest
0,
"snapshot1",
"keyspace1",
- "table1-1234",
+ "table1",
"1.db"),
new ListSnapshotFilesResponse.FileInfo(0,
"localhost",
@@ -158,7 +158,7 @@ public class SnapshotsHandlerTest
0,
"snapshot1",
"keyspace1",
- "table1-1234",
+ "table1",
".index/secondary.db")
);
ListSnapshotFilesResponse.FileInfo fileInfoNotExpected =
@@ -168,7 +168,7 @@ public class SnapshotsHandlerTest
0,
"snapshot1",
"keyspace1",
- "table1-1234",
+ "table1",
"2.db");
client.get(server.actualPort(), "localhost", testRoute)
@@ -185,7 +185,7 @@ public class SnapshotsHandlerTest
public void testRouteInvalidSnapshot(VertxTestContext context)
{
WebClient client = WebClient.create(vertx);
- String testRoute =
"/api/v1/keyspaces/keyspace1/tables/table1-1234/snapshots/snapshotInvalid";
+ String testRoute =
"/api/v1/keyspaces/keyspace1/tables/table1/snapshots/snapshotInvalid";
client.get(server.actualPort(), "localhost", testRoute)
.send(context.succeeding(response -> context.verify(() -> {
assertThat(response.statusCode()).isEqualTo(NOT_FOUND.code());
diff --git
a/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
b/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
index 48a0385..00f7c65 100644
---
a/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
+++
b/src/test/java/org/apache/cassandra/sidecar/routes/StreamSSTableComponentHandlerTest.java
@@ -59,7 +59,7 @@ public class StreamSSTableComponentHandlerTest
{
private static final Logger logger =
LoggerFactory.getLogger(StreamSSTableComponentHandlerTest.class);
static final String TEST_KEYSPACE = "TestKeyspace";
- static final String TEST_TABLE =
"TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b";
+ static final String TEST_TABLE = "TestTable";
private Vertx vertx;
private Server server;
@@ -95,7 +95,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.as(BodyCodec.buffer())
.send(context.succeeding(response -> context.verify(() -> {
@@ -140,7 +140,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/random/tables/" + TEST_TABLE +
"/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.send(context.succeeding(response -> context.verify(() -> {
assertThat(response.statusCode()).isEqualTo(NOT_FOUND.code());
@@ -153,7 +153,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/random/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/random/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.send(context.succeeding(response -> context.verify(() -> {
assertThat(response.statusCode()).isEqualTo(NOT_FOUND.code());
@@ -166,7 +166,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/system/tables/" + TEST_TABLE +
"/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.send(context.succeeding(response -> context.verify(() -> {
assertThat(response.statusCode()).isEqualTo(FORBIDDEN.code());
@@ -180,7 +180,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/k*s/tables/" + TEST_TABLE +
"/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.send(context.succeeding(response -> context.verify(() -> {
assertThat(response.statusCode()).isEqualTo(BAD_REQUEST.code());
@@ -268,7 +268,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE +
"/tables/TestTable/snapshots/TestSnapshot/components" +
- "/" + TEST_KEYSPACE + "-" + TEST_TABLE + "-Data.db";
+ "/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=0-")
.as(BodyCodec.buffer())
@@ -284,7 +284,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=4-3")
.send(context.succeeding(response -> context.verify(() -> {
@@ -298,7 +298,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=5-9")
.send(context.succeeding(response -> context.verify(() -> {
@@ -312,7 +312,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=5-")
.send(context.succeeding(response -> context.verify(() -> {
@@ -326,7 +326,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=0-999999")
.as(BodyCodec.buffer())
@@ -342,7 +342,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=0-2") // 3 bytes streamed
.as(BodyCodec.buffer())
@@ -358,7 +358,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=-2") // last 2 bytes streamed
.as(BodyCodec.buffer())
@@ -374,7 +374,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bytes=-5")
.send(context.succeeding(response -> context.verify(() -> {
@@ -391,7 +391,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/snapshots" +
- "/TestSnapshot/components/" + TEST_KEYSPACE + "-" +
TEST_TABLE + "-Data.db";
+ "/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute)
.putHeader("Range", "bits=0-2")
.send(context.succeeding(response -> context.verify(() -> {
@@ -405,8 +405,7 @@ public class StreamSSTableComponentHandlerTest
{
WebClient client = WebClient.create(vertx);
String testRoute = "/keyspaces/" + TEST_KEYSPACE + "/tables/" +
TEST_TABLE + "/" +
- "snapshots/TestSnapshot/components/" +
- TEST_KEYSPACE + "-" + TEST_TABLE + "-Data.db";
+
"snapshots/TestSnapshot/components/nb-1-big-Data.db";
client.get(server.actualPort(), "localhost", "/api/v1" + testRoute +
"?instanceId=2")
.as(BodyCodec.buffer())
.send(context.succeeding(response -> context.verify(() -> {
diff --git
a/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandlerTest.java
b/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandlerTest.java
index 26ca9dc..49554c6 100644
---
a/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandlerTest.java
+++
b/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableUploadHandlerTest.java
@@ -69,7 +69,7 @@ class SSTableUploadHandlerTest extends BaseUploadsHandlerTest
{
private static final String FILE_TO_BE_UPLOADED =
"./src/test/resources/instance1/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots"
- +
"/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db";
+ + "/TestSnapshot/nb-1-big-Data.db";
@Test
void testUploadWithoutMd5_expectSuccessfulUpload(VertxTestContext context)
throws IOException
diff --git
a/src/test/java/org/apache/cassandra/sidecar/utils/CassandraInputValidatorTest.java
b/src/test/java/org/apache/cassandra/sidecar/utils/CassandraInputValidatorTest.java
index 78e7105..8e47f19 100644
---
a/src/test/java/org/apache/cassandra/sidecar/utils/CassandraInputValidatorTest.java
+++
b/src/test/java/org/apache/cassandra/sidecar/utils/CassandraInputValidatorTest.java
@@ -21,11 +21,13 @@ package org.apache.cassandra.sidecar.utils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
import io.netty.handler.codec.http.HttpResponseStatus;
import io.vertx.ext.web.handler.HttpException;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.assertj.core.api.Assertions.assertThat;
/**
* Test validation methods.
@@ -40,69 +42,61 @@ public class CassandraInputValidatorTest
instance = new CassandraInputValidator();
}
- private void testCommon_invalidCharacters(String testName)
+ @ParameterizedTest(name = "[{0}]")
+ @ValueSource(strings = { "test_table_name", "\"test_table_name\"",
"testTableName", "\"testTableName\"", "a_",
+ "\"cycling\"", "\"Helmets\"", "\"mIxEd_cAsE\"",
"a8", "a", "\"8a\"",
+ "\"_must_begin_with_alphabetic_unless_quoted_p\""
})
+ public void testValidTableNameValidation(String tableName)
{
- HttpException httpEx = Assertions.assertThrows(HttpException.class, ()
-> instance.validateTableName(testName));
- assertEquals(HttpResponseStatus.BAD_REQUEST.code(),
httpEx.getStatusCode());
- assertEquals("Invalid characters in table name: " + testName,
httpEx.getPayload());
+ instance.validateTableName(tableName);
}
- @Test
- public void testValidateCharacters_validParams_expectNoException()
- {
- instance.validateTableName("test_table_name");
- instance.validateTableName("test-table-name");
- instance.validateTableName("testTableName");
- }
-
- @Test
- public void testValidateCharacters_paramWithColon_expectException()
+ @ParameterizedTest(name = "[{0}]")
+ @ValueSource(strings = { "", "test table", "_must_begin_with_alphabetic",
"dash-is-not-allowed", "\"\"", "\"",
+ "\"inv@lid_chars\"", "test:table_name",
"test-table$name", "8a", "testTable/Name" })
+ public void failsWithInvalidTableName(String tableName)
{
- testCommon_invalidCharacters("test:table_name");
- }
-
- @Test
- public void testValidateCharacters_paramWithDollar_expectException()
- {
- testCommon_invalidCharacters("test-table$name");
- }
-
- @Test
- public void testValidateCharacters_paramsWithSlash_expectException()
- {
- testCommon_invalidCharacters("testTable/Name");
+ HttpException httpEx = Assertions.assertThrows(HttpException.class,
+ () ->
instance.validateTableName(tableName));
+
assertThat(httpEx.getStatusCode()).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
+ assertThat(httpEx.getPayload()).isEqualTo("Invalid characters in table
name: " + tableName);
}
-
- @Test
- public void testValidateKeyspaceName_validKeyspaceNames_expectNoException()
+ @ParameterizedTest(name = "[{0}]")
+ @ValueSource(strings = { "SystemViews", "system_views_test",
"\"keyspace\"", "\"cycling\"", "\"Cycling\"",
+ "\"mIxEd_cAsE\"", "a8", "a", "a_", "\"_a\"" })
+ public void testValidKeyspaceValidation(String keyspace)
{
- instance.validateKeyspaceName("system-views");
- instance.validateKeyspaceName("SystemViews");
- instance.validateKeyspaceName("system_views_test");
+ instance.validateKeyspaceName(keyspace);
}
- @Test
- public void
testValidateKeyspaceName_forbiddenKeyspaceName_expectException()
+ @ParameterizedTest(name = "[{0}]")
+ @ValueSource(strings = { "system_schema",
+ "system_traces",
+ "system_distributed",
+ "system",
+ "system_auth",
+ "system_views",
+ "system_virtual_schema" })
+ public void failsWithForbiddenKeyspace(String keyspace)
{
- String testKS = "system_views";
HttpException httpEx = Assertions.assertThrows(HttpException.class,
- () ->
instance.validateKeyspaceName(testKS));
- assertEquals(HttpResponseStatus.FORBIDDEN.code(),
httpEx.getStatusCode());
- assertEquals("Forbidden keyspace: " + testKS, httpEx.getPayload());
+ () ->
instance.validateKeyspaceName(keyspace));
+
assertThat(httpEx.getStatusCode()).isEqualTo(HttpResponseStatus.FORBIDDEN.code());
+ assertThat(httpEx.getPayload()).isEqualTo("Forbidden keyspace: " +
keyspace);
}
- @Test
- public void
testValidateKeyspaceName_keyspaceNameWithSpace_expectException()
+ @ParameterizedTest(name = "[{0}]")
+ @ValueSource(strings = { "", "test keyspace", "_cycling",
"dash-is-not-allowed", "\"\"", "\"",
+ "\"inv@lid_chars\"", "8a" })
+ public void failsWithInvalidKeyspaceName(String keyspace)
{
- String testKS = "test keyspace";
HttpException httpEx = Assertions.assertThrows(HttpException.class,
- () ->
instance.validateKeyspaceName(testKS));
- assertEquals(HttpResponseStatus.BAD_REQUEST.code(),
httpEx.getStatusCode());
- assertEquals("Invalid characters in keyspace: " + testKS,
httpEx.getPayload());
+ () ->
instance.validateKeyspaceName(keyspace));
+
assertThat(httpEx.getStatusCode()).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
+ assertThat(httpEx.getPayload()).isEqualTo("Invalid characters in
keyspace: " + keyspace);
}
-
@Test
public void testValidateFileName_validFileNames_expectNoException()
{
@@ -117,8 +111,8 @@ public class CassandraInputValidatorTest
{
HttpException httpEx = Assertions.assertThrows(HttpException.class,
() ->
instance.validateComponentName(testFileName));
- assertEquals(HttpResponseStatus.BAD_REQUEST.code(),
httpEx.getStatusCode());
- assertEquals("Invalid component name: " + testFileName,
httpEx.getPayload());
+
assertThat(httpEx.getStatusCode()).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
+ assertThat(httpEx.getPayload()).isEqualTo("Invalid component name: " +
testFileName);
}
@Test
@@ -162,8 +156,8 @@ public class CassandraInputValidatorTest
String testSnapName = "valid" + '/' + "snapshotname";
HttpException httpEx = Assertions.assertThrows(HttpException.class,
() ->
instance.validateSnapshotName(testSnapName));
- assertEquals(HttpResponseStatus.BAD_REQUEST.code(),
httpEx.getStatusCode());
- assertEquals("Invalid characters in snapshot name: " + testSnapName,
httpEx.getPayload());
+
assertThat(httpEx.getStatusCode()).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
+ assertThat(httpEx.getPayload()).isEqualTo("Invalid characters in
snapshot name: " + testSnapName);
}
@Test
@@ -172,7 +166,7 @@ public class CassandraInputValidatorTest
String testSnapName = "valid" + '\0' + "snapshotname";
HttpException httpEx = Assertions.assertThrows(HttpException.class,
() ->
instance.validateSnapshotName(testSnapName));
- assertEquals(HttpResponseStatus.BAD_REQUEST.code(),
httpEx.getStatusCode());
- assertEquals("Invalid characters in snapshot name: " + testSnapName,
httpEx.getPayload());
+
assertThat(httpEx.getStatusCode()).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
+ assertThat(httpEx.getPayload()).isEqualTo("Invalid characters in
snapshot name: " + testSnapName);
}
}
diff --git a/src/test/resources/config/sidecar_multiple_instances.yaml
b/src/test/resources/config/sidecar_multiple_instances.yaml
index c51cea0..72fc1df 100644
--- a/src/test/resources/config/sidecar_multiple_instances.yaml
+++ b/src/test/resources/config/sidecar_multiple_instances.yaml
@@ -134,6 +134,7 @@ cassandra_input_validation:
- system_auth
- system_views
- system_virtual_schema
- allowed_chars_for_directory: "[a-zA-Z0-9_-]+"
+ allowed_chars_for_directory: "[a-zA-Z][a-zA-Z0-9_]{0,47}"
+ allowed_chars_for_quoted_name: "[a-zA-Z_0-9]{1,48}"
allowed_chars_for_component_name:
"[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)"
allowed_chars_for_restricted_component_name: "[a-zA-Z0-9_-]+(.db|TOC.txt)"
diff --git a/src/test/resources/config/sidecar_single_instance.yaml
b/src/test/resources/config/sidecar_single_instance.yaml
index f95fd57..64c9bfd 100644
--- a/src/test/resources/config/sidecar_single_instance.yaml
+++ b/src/test/resources/config/sidecar_single_instance.yaml
@@ -87,6 +87,7 @@ cassandra_input_validation:
- system_auth
- system_views
- system_virtual_schema
- allowed_chars_for_directory: "[a-zA-Z0-9_-]+"
+ allowed_chars_for_directory: "[a-zA-Z][a-zA-Z0-9_]{0,47}"
+ allowed_chars_for_quoted_name: "[a-zA-Z_0-9]{1,48}"
allowed_chars_for_component_name:
"[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)"
allowed_chars_for_restricted_component_name: "[a-zA-Z0-9_-]+(.db|TOC.txt)"
diff --git a/src/test/resources/config/sidecar_ssl.yaml
b/src/test/resources/config/sidecar_ssl.yaml
index 644f359..afe7731 100644
--- a/src/test/resources/config/sidecar_ssl.yaml
+++ b/src/test/resources/config/sidecar_ssl.yaml
@@ -134,6 +134,7 @@ cassandra_input_validation:
- system_auth
- system_views
- system_virtual_schema
- allowed_chars_for_directory: "[a-zA-Z0-9_-]+"
+ allowed_chars_for_directory: "[a-zA-Z][a-zA-Z0-9_]{0,47}"
+ allowed_chars_for_quoted_name: "[a-zA-Z_0-9]{1,48}"
allowed_chars_for_component_name:
"[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)"
allowed_chars_for_restricted_component_name: "[a-zA-Z0-9_-]+(.db|TOC.txt)"
diff --git a/src/test/resources/config/sidecar_validation_configuration.yaml
b/src/test/resources/config/sidecar_validation_configuration.yaml
index 63ffe86..2d6364f 100644
--- a/src/test/resources/config/sidecar_validation_configuration.yaml
+++ b/src/test/resources/config/sidecar_validation_configuration.yaml
@@ -30,5 +30,6 @@ cassandra_input_validation:
- b
- c
allowed_chars_for_directory: "[a-z]+"
+ allowed_chars_for_quoted_name: "[A-Z]+"
allowed_chars_for_component_name: "(.db|.cql|.json|.crc32|TOC.txt)"
allowed_chars_for_restricted_component_name: "(.db|TOC.txt)"
diff --git
a/src/test/resources/instance1/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db
b/src/test/resources/instance1/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/nb-1-big-Data.db
similarity index 100%
rename from
src/test/resources/instance1/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db
rename to
src/test/resources/instance1/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/nb-1-big-Data.db
diff --git
a/src/test/resources/instance2/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db
b/src/test/resources/instance2/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/nb-1-big-Data.db
similarity index 100%
rename from
src/test/resources/instance2/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db
rename to
src/test/resources/instance2/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/nb-1-big-Data.db
diff --git
a/src/test/resources/instance3/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db
b/src/test/resources/instance3/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/nb-1-big-Data.db
similarity index 100%
rename from
src/test/resources/instance3/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/TestKeyspace-TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b-Data.db
rename to
src/test/resources/instance3/data/TestKeyspace/TestTable-54ea95ce-bba2-4e0a-a9be-e428e5d7160b/snapshots/TestSnapshot/nb-1-big-Data.db
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]