This is an automated email from the ASF dual-hosted git repository.
frankgh pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-sidecar.git
The following commit(s) were added to refs/heads/trunk by this push:
new f41b3ca8 CASSSIDECAR-272: Add /api/v2/cassandra/settings which will
return Cassandra configurations stored in system_views.settings (#264)
f41b3ca8 is described below
commit f41b3ca8cc8b3f89fb5f5fa5ba760c42fd4df8e0
Author: Isaac Reath <[email protected]>
AuthorDate: Mon Nov 17 16:24:45 2025 -0500
CASSSIDECAR-272: Add /api/v2/cassandra/settings which will return Cassandra
configurations stored in system_views.settings (#264)
patch by Isaac Reath; reviewed by Francisco Guerrero, Saranya Krishnakumar
for CASSSIDECAR-272
---
CHANGES.txt | 1 +
.../sidecar/adapters/base/CassandraAdapter.java | 8 +
.../cassandra/sidecar/common/ApiEndpointsV2.java | 29 ++++
.../sidecar/common/response/v2/V2NodeSettings.java | 65 ++++++++
.../routes/v2/V2NodeSettingsIntegrationTest.java | 116 +++++++++++++++
.../sidecar/common/server/ICassandraAdapter.java | 8 +
.../cassandra/sidecar/CassandraSidecarDaemon.java | 6 +-
.../sidecar/cluster/CassandraAdapterDelegate.java | 47 +++++-
.../sidecar/db/schema/SystemViewsSchema.java | 10 +-
.../v2/cassandra/V2NodeSettingsHandler.java | 82 ++++++++++
.../sidecar/modules/CassandraOperationsModule.java | 19 +++
.../modules/multibindings/VertxRouteMapKeys.java | 6 +
.../cluster/CassandraAdapterDelegateTest.java | 165 +++++++++++++++++++++
.../cassandra/sidecar/db/SidecarSchemaTest.java | 2 +
.../v2/cassandra/V2NodeSettingsHandlerTest.java | 95 ++++++++++++
15 files changed, 650 insertions(+), 9 deletions(-)
diff --git a/CHANGES.txt b/CHANGES.txt
index 5d82fef6..c53846b7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
0.3.0
-----
+ * Add /api/v2/cassandra/settings which will return Cassandra configurations
stored in system_views.settings (CASSSIDECAR-272)
* Fast Cassandra Input Validator (CASSSIDECAR-361)
* Upgrade caffeine dependency (CASSSIDECAR-332)
* Add Live Migration Status endpoint to persist Live Migration status and for
safety (CASSSIDECAR-345)
diff --git
a/adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
b/adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
index 15f5a73e..556c405a 100644
---
a/adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
+++
b/adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
@@ -19,6 +19,7 @@
package org.apache.cassandra.sidecar.adapters.base;
import java.net.InetSocketAddress;
+import java.util.Map;
import java.util.Objects;
import com.datastax.driver.core.ConsistencyLevel;
@@ -104,6 +105,13 @@ public class CassandraAdapter implements ICassandraAdapter
throw new UnsupportedOperationException("Node settings are not
provided by this adapter");
}
+ @Override
+ @NotNull
+ public Map<String, String> v2NodeSettings()
+ {
+ throw new UnsupportedOperationException("V2 node settings are not
provided by this adapter");
+ }
+
@Override
@NotNull
public ResultSet executeLocal(Statement statement)
diff --git
a/client-common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV2.java
b/client-common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV2.java
new file mode 100644
index 00000000..b85445ce
--- /dev/null
+++
b/client-common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV2.java
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * A constants container class for API endpoints of version 2.
+ */
+public class ApiEndpointsV2
+{
+ public static final String API = "/api";
+ public static final String API_V2 = API + "/v2";
+ public static final String CASSANDRA = "/cassandra";
+ public static final String NODE_SETTINGS_ROUTE = API_V2 + CASSANDRA +
"/settings";
+}
diff --git
a/client-common/src/main/java/org/apache/cassandra/sidecar/common/response/v2/V2NodeSettings.java
b/client-common/src/main/java/org/apache/cassandra/sidecar/common/response/v2/V2NodeSettings.java
new file mode 100644
index 00000000..9f7bd24a
--- /dev/null
+++
b/client-common/src/main/java/org/apache/cassandra/sidecar/common/response/v2/V2NodeSettings.java
@@ -0,0 +1,65 @@
+/*
+ * 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.response.v2;
+
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.cassandra.sidecar.common.response.NodeSettings;
+
+/**
+ * V2NodeSettings stores settings information for Cassandra.
+ */
+public class V2NodeSettings
+{
+ // Contains Cassandra node settings from system_views.setting table.
+ private final Map<String, String> nodeSettings;
+
+ /**
+ * Constructs a new {@link NodeSettings}.
+ */
+
+ @JsonCreator
+ public V2NodeSettings(@JsonProperty("nodeSettings") Map<String, String>
nodeSettings)
+ {
+ this.nodeSettings = nodeSettings;
+ }
+
+ @JsonProperty("nodeSettings")
+ public Map<String, String> nodeSettings()
+ {
+ return nodeSettings;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (o == null || getClass() != o.getClass()) return false;
+ V2NodeSettings that = (V2NodeSettings) o;
+ return Objects.equals(nodeSettings, that.nodeSettings);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(nodeSettings);
+ }
+
+}
diff --git
a/integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/v2/V2NodeSettingsIntegrationTest.java
b/integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/v2/V2NodeSettingsIntegrationTest.java
new file mode 100644
index 00000000..b1ea746c
--- /dev/null
+++
b/integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/v2/V2NodeSettingsIntegrationTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.v2;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import io.vertx.core.VertxException;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.core.http.HttpResponseExpectation;
+import io.vertx.ext.web.client.HttpResponse;
+import org.apache.cassandra.sidecar.common.response.v2.V2NodeSettings;
+import
org.apache.cassandra.sidecar.testing.SharedClusterSidecarIntegrationTestBase;
+
+import static
io.netty.handler.codec.http.HttpResponseStatus.SERVICE_UNAVAILABLE;
+import static org.apache.cassandra.testing.utils.AssertionUtils.getBlocking;
+import static org.apache.cassandra.testing.utils.AssertionUtils.loopAssert;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * V2NodeSettingsIntegrationTest is responsible for verifying the behavior of
the /api/v2/cassandra/settings
+ * endpoint. This includes:
+ * - Node settings are returned when the node is healthy.
+ * - A specific error is returned when CQL is not healthy.
+ * - A separate error is returned when the node is down.
+ */
+public class V2NodeSettingsIntegrationTest extends
SharedClusterSidecarIntegrationTestBase
+{
+
+ @Test
+ public void testV2NodeSettings()
+ {
+ // Start by setting a known configuration for concurrent_reads
+ String concurrencyKey = "concurrent_reads";
+ String expectedValue = "20";
+ cluster.getFirstRunningInstance().nodetool("setconcurrency", "READ",
expectedValue);
+ ensureSettingsAvailable(concurrencyKey, expectedValue);
+
+ // Disabling NTR should make CQL settings become unavailable
+ cluster.getFirstRunningInstance().nodetool("disablebinary");
+ ensureSettingsBecomeUnavailable("CQL NodeSettings unavailable");
+
+ // Re-enable NTR, settings should become available again. The value of
concurrent_reads should not change.
+ cluster.getFirstRunningInstance().nodetool("enablebinary");
+ ensureSettingsAvailable(concurrencyKey, expectedValue);
+
+ // Changing a configuration should eventually reflect in settings API.
+ expectedValue = "10";
+ cluster.getFirstRunningInstance().nodetool("setconcurrency", "READ",
expectedValue);
+ ensureSettingsAvailable(concurrencyKey, expectedValue);
+
+ cluster.stopUnchecked(cluster.getFirstRunningInstance());
+ ensureSettingsBecomeUnavailable("NodeSettings unavailable");
+ }
+
+ private void ensureSettingsBecomeUnavailable(String errorMessage)
+ {
+ loopAssert(60, () -> {
+ HttpResponse<Buffer> responseAfterStop =
getBlocking(trustedClient().get(serverWrapper.serverPort, "localhost",
"/api/v2/cassandra/settings")
+
.send()
+
.expecting(HttpResponseExpectation.SC_SERVICE_UNAVAILABLE));
+ assertThat(responseAfterStop).isNotNull();
+
assertThat(responseAfterStop.statusCode()).isEqualTo(SERVICE_UNAVAILABLE.code());
+
assertThat(responseAfterStop.bodyAsJsonObject().getString("message")).contains(errorMessage);
+ });
+ }
+
+ private void ensureSettingsAvailable(String expectedSettingKey, String
expectedSettingValue)
+ {
+ loopAssert(60, () -> {
+ HttpResponse<Buffer> response = null;
+ try
+ {
+ response =
getBlocking(trustedClient().get(serverWrapper.serverPort, "localhost",
"/api/v2/cassandra/settings")
+ .send()
+
.expecting(HttpResponseExpectation.SC_OK));
+ }
+ catch (VertxException e)
+ {
+ Assertions.fail(e);
+ }
+ V2NodeSettings nodeSettings =
response.bodyAsJson(V2NodeSettings.class);
+ assertThat(nodeSettings).isNotNull();
+ Map<String, String> cqlSettings = new HashMap<>();
+ cluster.getFirstRunningInstance()
+ .executeInternalWithResult("SELECT name, value FROM
system_views.settings;")
+ .forEach(row -> cqlSettings.put(row.getString("name"),
row.getString("value")));
+ assertThat(nodeSettings.nodeSettings()).isEqualTo(cqlSettings);
+
assertThat(cqlSettings.get(expectedSettingKey)).isEqualTo(expectedSettingValue);
+ });
+ }
+
+ @Override
+ protected void initializeSchemaForTest()
+ {
+ // Do nothing
+ }
+}
diff --git
a/server-common/src/main/java/org/apache/cassandra/sidecar/common/server/ICassandraAdapter.java
b/server-common/src/main/java/org/apache/cassandra/sidecar/common/server/ICassandraAdapter.java
index c18ee91e..59be199f 100644
---
a/server-common/src/main/java/org/apache/cassandra/sidecar/common/server/ICassandraAdapter.java
+++
b/server-common/src/main/java/org/apache/cassandra/sidecar/common/server/ICassandraAdapter.java
@@ -19,6 +19,7 @@
package org.apache.cassandra.sidecar.common.server;
import java.net.InetSocketAddress;
+import java.util.Map;
import com.datastax.driver.core.Metadata;
import com.datastax.driver.core.ResultSet;
@@ -51,6 +52,13 @@ public interface ICassandraAdapter
*/
@NotNull NodeSettings nodeSettings() throws CassandraUnavailableException;
+ /**
+ * The settings for this instance which are stored in the
system_view.settings virtual table.
+ * @return A map of name: value mappings for the settings in
system_view.settings.
+ * @throws CassandraUnavailableException when CQL is not available
+ */
+ @NotNull Map<String, String> v2NodeSettings() throws
CassandraUnavailableException;
+
/**
* Execute the provided query on the locally-managed Cassandra instance
*
diff --git
a/server/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
b/server/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
index cb1479e3..ddf3c24d 100644
---
a/server/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
+++
b/server/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
@@ -60,7 +60,7 @@ public class CassandraSidecarDaemon
}
/**
- * Closes the server, waits up to 1 minute for the server to shut down.
+ * Closes the server, waits up to 2 minute for the server to shut down.
*
* @param app the server
* @return {@code true} if the server shutdown successfully, {@code false}
otherwise
@@ -73,12 +73,12 @@ public class CassandraSidecarDaemon
app.close()
.toCompletionStage()
.toCompletableFuture()
- .get(1, TimeUnit.MINUTES);
+ .get(2, TimeUnit.MINUTES);
return true;
}
catch (Exception ex)
{
- LOGGER.warn("Failed to stop Sidecar in 1 minute", ex);
+ LOGGER.warn("Failed to stop Sidecar in 2 minutes", ex);
}
return false;
}
diff --git
a/server/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
b/server/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
index 4b5655bf..481693d1 100644
---
a/server/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
+++
b/server/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
@@ -21,8 +21,10 @@ package org.apache.cassandra.sidecar.cluster;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collections;
+import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.List;
+import java.util.Map;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Function;
@@ -65,6 +67,7 @@ import org.jetbrains.annotations.NotNull;
import static
org.apache.cassandra.sidecar.adapters.base.jmx.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
import static
org.apache.cassandra.sidecar.adapters.base.jmx.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static
org.apache.cassandra.sidecar.exceptions.CassandraUnavailableException.Service.CQL;
import static
org.apache.cassandra.sidecar.exceptions.CassandraUnavailableException.Service.CQL_AND_JMX;
import static
org.apache.cassandra.sidecar.exceptions.CassandraUnavailableException.Service.JMX;
import static
org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CASSANDRA_CQL_DISCONNECTED;
@@ -98,6 +101,7 @@ public class CassandraAdapterDelegate implements
ICassandraAdapter, Host.StateLi
private volatile ICassandraAdapter adapter;
private final AtomicBoolean isNativeUp = new AtomicBoolean(false);
private volatile NodeSettings nodeSettingsFromJmx = null;
+ private volatile Map<String, String> nodeSettingsFromCql = null;
private final AtomicBoolean registered = new AtomicBoolean(false);
private final AtomicBoolean isHealthCheckActive = new AtomicBoolean(false);
private final InetSocketAddress localNativeTransportAddress;
@@ -262,20 +266,19 @@ public class CassandraAdapterDelegate implements
ICassandraAdapter, Host.StateLi
try
{
// NOTE: We cannot use `executeLocal` here as there may be no
adapter yet.
- SimpleStatement healthCheckStatement = new SimpleStatement("SELECT
release_version FROM system.local");
Metadata metadata = activeSession.getCluster().getMetadata();
host = getHost(metadata);
if (host == null)
{
LOGGER.warn("Could not find host in cluster metadata by
address and port {}",
localNativeTransportAddress);
+ nodeSettingsFromCql = null;
return;
}
- healthCheckStatement.setHost(host);
- healthCheckStatement.setConsistencyLevel(ConsistencyLevel.ONE);
- Row row = activeSession.execute(healthCheckStatement).one();
+ nodeSettingsFromCql = queryNodeSettingsFromCql(activeSession,
host);
// This should never happen but added for completeness
- Preconditions.checkArgument(row != null, "Session execution result
should never be null");
+ Preconditions.checkArgument(nodeSettingsFromCql != null &&
+ !nodeSettingsFromCql.isEmpty(),
"Session execution result should never be null");
if (isNativeUp.compareAndSet(false, true))
{
@@ -289,6 +292,7 @@ public class CassandraAdapterDelegate implements
ICassandraAdapter, Host.StateLi
markNativeDownAndMaybeNotifyDisconnection();
// Unregister the host listener.
maybeUnregisterHostListener(activeSession);
+ nodeSettingsFromCql = null;
}
}
@@ -380,6 +384,25 @@ public class CassandraAdapterDelegate implements
ICassandraAdapter, Host.StateLi
return nodeSettingsFromJmx;
}
+ /**
+ * Returns the cached node settings read from the system_views.settings
table. These are retrieved after a successful
+ * health check for native transport.
+ *
+ * @return Cached node settings from system_view.settings.
+ * @throws CassandraUnavailableException Thrown when native transport is
not available.
+ */
+ @Override
+ @NotNull
+ public Map<String, String> v2NodeSettings() throws
CassandraUnavailableException
+ {
+ Map<String, String> settings = nodeSettingsFromCql;
+ if (settings == null)
+ {
+ throw new CassandraUnavailableException(CQL, "CQL NodeSettings
unavailable");
+ }
+ return settings;
+ }
+
@Override
@NotNull
public ResultSet executeLocal(Statement statement) throws
CassandraUnavailableException
@@ -573,6 +596,20 @@ public class CassandraAdapterDelegate implements
ICassandraAdapter, Host.StateLi
}
}
+ protected Map<String, String> queryNodeSettingsFromCql(Session
activeSession, Host host)
+ {
+ SimpleStatement allSystemSettingsStatement = new
SimpleStatement("SELECT name, value FROM system_views.settings");
+ allSystemSettingsStatement.setHost(host);
+ allSystemSettingsStatement.setConsistencyLevel(ConsistencyLevel.ONE);
+ ResultSet result = activeSession.execute(allSystemSettingsStatement);
+ Map<String, String> nodeSettings = new HashMap<>();
+ for (Row setting : result.all())
+ {
+ nodeSettings.put(setting.getString("name"),
setting.getString("value"));
+ }
+ return Collections.unmodifiableMap(nodeSettings);
+ }
+
@NotNull
private <T> T fromAdapter(Function<ICassandraAdapter, T> getter) throws
CassandraUnavailableException
{
diff --git
a/server/src/main/java/org/apache/cassandra/sidecar/db/schema/SystemViewsSchema.java
b/server/src/main/java/org/apache/cassandra/sidecar/db/schema/SystemViewsSchema.java
index e640f42a..fddf0356 100644
---
a/server/src/main/java/org/apache/cassandra/sidecar/db/schema/SystemViewsSchema.java
+++
b/server/src/main/java/org/apache/cassandra/sidecar/db/schema/SystemViewsSchema.java
@@ -33,6 +33,7 @@ public class SystemViewsSchema extends
CassandraSystemTableSchema
protected static final String SYSTEM_VIEWS_KEYSPACE_NAME = "system_views";
protected static final String SYSTEM_VIEWS_SETTINGS_TABLE_NAME =
"settings";
private PreparedStatement selectSettings;
+ private PreparedStatement selectAllSettings;
@Override
protected String keyspaceName()
@@ -52,15 +53,22 @@ public class SystemViewsSchema extends
CassandraSystemTableSchema
return selectSettings;
}
+ public PreparedStatement selectAllSettings() throws
SchemaUnavailableException
+ {
+ ensureSchemaAvailable();
+ return selectAllSettings;
+ }
+
@Override
protected void prepareStatements(@NotNull Session session)
{
this.selectSettings = session.prepare("SELECT name, value FROM
system_views.settings WHERE name IN ?");
+ this.selectAllSettings = session.prepare("SELECT name, value FROM
system_views.settings");
}
protected void ensureSchemaAvailable() throws SchemaUnavailableException
{
- if (selectSettings == null)
+ if (selectSettings == null || selectAllSettings == null)
{
throw new SchemaUnavailableException(keyspaceName(), tableName());
}
diff --git
a/server/src/main/java/org/apache/cassandra/sidecar/handlers/v2/cassandra/V2NodeSettingsHandler.java
b/server/src/main/java/org/apache/cassandra/sidecar/handlers/v2/cassandra/V2NodeSettingsHandler.java
new file mode 100644
index 00000000..e0881644
--- /dev/null
+++
b/server/src/main/java/org/apache/cassandra/sidecar/handlers/v2/cassandra/V2NodeSettingsHandler.java
@@ -0,0 +1,82 @@
+/*
+ * 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.handlers.v2.cassandra;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.inject.Inject;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.auth.authorization.Authorization;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.cassandra.sidecar.acl.authorization.CassandraPermissions;
+import org.apache.cassandra.sidecar.common.response.v2.V2NodeSettings;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.handlers.AbstractHandler;
+import org.apache.cassandra.sidecar.handlers.AccessProtected;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+import org.jetbrains.annotations.NotNull;
+
+import static
org.apache.cassandra.sidecar.acl.authorization.DataResourceScope.DATA_SCOPE;
+
+/**
+ * V2NodeSettingsHandler is responsible for providing access to the
configurations of the
+ * Cassandra instances managed by this sidecar. This includes settings
accessible via CQL from the system_views.settings table.
+ */
+public class V2NodeSettingsHandler extends AbstractHandler<Void> implements
AccessProtected
+{
+
+ /**
+ * Constructs a handler with the provided {@code metadataFetcher}
+ *
+ * @param metadataFetcher the interface to retrieve instance metadata
+ */
+ @Inject
+ V2NodeSettingsHandler(InstanceMetadataFetcher metadataFetcher,
ExecutorPools executorPools)
+ {
+ super(metadataFetcher, executorPools, null);
+ }
+
+ @Override
+ protected Void extractParamsOrThrow(RoutingContext context)
+ {
+ return null;
+ }
+
+ @Override
+ protected void handleInternal(RoutingContext context, HttpServerRequest
httpRequest, @NotNull String host, SocketAddress remoteAddress, Void request)
+ {
+ Map<String, String> cqlSettings =
metadataFetcher.delegate(host).v2NodeSettings();
+ V2NodeSettings v2nodeSettings = new V2NodeSettings(cqlSettings);
+ context.json(v2nodeSettings);
+ }
+
+ @Override
+ public Set<Authorization> requiredAuthorizations()
+ {
+ Set<String> eligibleResources =
Set.of(DATA_SCOPE.variableAwareResource(),
+ // Keyspace access to
system_views
+ "data/system_views",
+ // Access to all tables in
keyspace system_views
+ "data/system_views/*",
+ // Access to the settings table
in the system_views keyspace
+ "data/system_views/settings");
+ return
Set.of(CassandraPermissions.SELECT.toAuthorization(eligibleResources));
+ }
+}
diff --git
a/server/src/main/java/org/apache/cassandra/sidecar/modules/CassandraOperationsModule.java
b/server/src/main/java/org/apache/cassandra/sidecar/modules/CassandraOperationsModule.java
index 3101f421..ebf94af9 100644
---
a/server/src/main/java/org/apache/cassandra/sidecar/modules/CassandraOperationsModule.java
+++
b/server/src/main/java/org/apache/cassandra/sidecar/modules/CassandraOperationsModule.java
@@ -25,6 +25,7 @@ import jakarta.ws.rs.PUT;
import jakarta.ws.rs.Path;
import
org.apache.cassandra.sidecar.adapters.base.db.schema.ConnectedClientsSchema;
import org.apache.cassandra.sidecar.common.ApiEndpointsV1;
+import org.apache.cassandra.sidecar.common.ApiEndpointsV2;
import org.apache.cassandra.sidecar.common.response.CompactionStatsResponse;
import
org.apache.cassandra.sidecar.common.response.ConnectedClientStatsResponse;
import org.apache.cassandra.sidecar.common.response.GossipInfoResponse;
@@ -35,6 +36,7 @@ import
org.apache.cassandra.sidecar.common.response.SchemaResponse;
import org.apache.cassandra.sidecar.common.response.StreamStatsResponse;
import org.apache.cassandra.sidecar.common.response.TableStatsResponse;
import org.apache.cassandra.sidecar.common.response.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.response.v2.V2NodeSettings;
import org.apache.cassandra.sidecar.db.schema.TableSchema;
import org.apache.cassandra.sidecar.handlers.CompactionStatsHandler;
import org.apache.cassandra.sidecar.handlers.ConnectedClientStatsHandler;
@@ -53,6 +55,7 @@ import
org.apache.cassandra.sidecar.handlers.StreamStatsHandler;
import org.apache.cassandra.sidecar.handlers.TableStatsHandler;
import org.apache.cassandra.sidecar.handlers.TokenRangeReplicaMapHandler;
import org.apache.cassandra.sidecar.handlers.cassandra.NodeSettingsHandler;
+import
org.apache.cassandra.sidecar.handlers.v2.cassandra.V2NodeSettingsHandler;
import
org.apache.cassandra.sidecar.handlers.validations.ValidateTableExistenceHandler;
import org.apache.cassandra.sidecar.modules.multibindings.KeyClassMapKey;
import org.apache.cassandra.sidecar.modules.multibindings.TableSchemaMapKeys;
@@ -214,6 +217,22 @@ public class CassandraOperationsModule extends
AbstractModule
.build();
}
+
+ @GET
+ @Path(ApiEndpointsV2.NODE_SETTINGS_ROUTE)
+ @Operation(summary = "Get node settings",
+ description = "Returns configuration settings for the
Cassandra node.")
+ @APIResponse(description = "Node settings retrieved successfully",
+ responseCode = "200",
+ content = @Content(mediaType = "application/json",
+ schema = @Schema(implementation = V2NodeSettings.class)))
+ @ProvidesIntoMap
+ @KeyClassMapKey(VertxRouteMapKeys.V2CassandraNodeSettingsRouteKey.class)
+ VertxRoute v2CassandraNodeSettings(RouteBuilder.Factory factory,
V2NodeSettingsHandler v2NodeSettingsHandler)
+ {
+ return factory.buildRouteWithHandler(v2NodeSettingsHandler);
+ }
+
@GET
@Path(ApiEndpointsV1.ALL_KEYSPACES_SCHEMA_ROUTE)
@Operation(summary = "Get all keyspaces schema",
diff --git
a/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/VertxRouteMapKeys.java
b/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/VertxRouteMapKeys.java
index 0657ab84..bc902ad9 100644
---
a/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/VertxRouteMapKeys.java
+++
b/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/VertxRouteMapKeys.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.sidecar.modules.multibindings;
import io.vertx.core.http.HttpMethod;
import org.apache.cassandra.sidecar.common.ApiEndpointsV1;
+import org.apache.cassandra.sidecar.common.ApiEndpointsV2;
/**
* Class keys in the {@link com.google.inject.multibindings.MapBinder} to
{@link org.apache.cassandra.sidecar.routes.VertxRoute} objects
@@ -338,4 +339,9 @@ public interface VertxRouteMapKeys
HttpMethod HTTP_METHOD = HttpMethod.PUT;
String ROUTE_URI = ApiEndpointsV1.SERVICE_CONFIG_ROUTE;
}
+ interface V2CassandraNodeSettingsRouteKey extends RouteClassKey
+ {
+ HttpMethod HTTP_METHOD = HttpMethod.GET;
+ String ROUTE_URI = ApiEndpointsV2.NODE_SETTINGS_ROUTE;
+ }
}
diff --git
a/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegateTest.java
b/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegateTest.java
new file mode 100644
index 00000000..86acf0f8
--- /dev/null
+++
b/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegateTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.cluster;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import com.codahale.metrics.MetricRegistry;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Host;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import io.vertx.core.Vertx;
+import io.vertx.core.eventbus.EventBus;
+import org.apache.cassandra.sidecar.common.server.CQLSessionProvider;
+import org.apache.cassandra.sidecar.common.server.JmxClient;
+import org.apache.cassandra.sidecar.common.server.utils.DriverUtils;
+import org.apache.cassandra.sidecar.exceptions.CassandraUnavailableException;
+import org.apache.cassandra.sidecar.metrics.instance.InstanceHealthMetrics;
+import org.apache.cassandra.sidecar.utils.CassandraVersionProvider;
+import org.jetbrains.annotations.NotNull;
+import org.mockito.Mockito;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.when;
+
+/**
+ * The CassandraAdapterDelegateTest is responsible for testing the
functionality of the CassandraAdapterDelegate with
+ * isolated dependencies.
+ */
+public class CassandraAdapterDelegateTest
+{
+ private CassandraAdapterDelegate cassandraAdapterDelegate;
+
+ @BeforeEach
+ public void setUp()
+ {
+ Vertx mockVertx = getMockVertx();
+ int cassandraInstanceId = 0;
+ CassandraVersionProvider cassandraVersionProvider =
Mockito.mock(CassandraVersionProvider.class);
+ Metadata metadata = Mockito.mock(Metadata.class);
+ CQLSessionProvider cqlSessionProvider =
getMockCqlSessionProvider(metadata);
+ JmxClient jmxClient = Mockito.mock(JmxClient.class);
+ String host = "localhost";
+ int port = 9042;
+ DriverUtils driverUtils = getMockDriverUtils(host, port, metadata);
+ String sidecarVersion = "0.2.0-SNAPSHOT";
+ InstanceHealthMetrics instanceHealthMetrics = new
InstanceHealthMetrics(new MetricRegistry());
+ cassandraAdapterDelegate = new CassandraAdapterDelegate(mockVertx,
+
cassandraInstanceId,
+
cassandraVersionProvider,
+
cqlSessionProvider,
+ jmxClient,
+ driverUtils,
+ sidecarVersion,
+ host,
+ port,
+
instanceHealthMetrics
+ );
+ }
+
+ private static @NotNull DriverUtils getMockDriverUtils(String host, int
port, Metadata metadata)
+ {
+ DriverUtils driverUtils = Mockito.mock(DriverUtils.class);
+ Host mockHost = Mockito.mock(Host.class);
+ InetSocketAddress mockAddress = new InetSocketAddress(host, port);
+ when(driverUtils.getHost(metadata, mockAddress)).thenReturn(mockHost);
+ return driverUtils;
+ }
+
+ private static @NotNull CQLSessionProvider
getMockCqlSessionProvider(Metadata metadata)
+ {
+ Session session = Mockito.mock(Session.class);
+ Cluster cluster = Mockito.mock(Cluster.class);
+ when(cluster.isClosed()).thenReturn(false);
+ when(cluster.getMetadata()).thenReturn(metadata);
+ when(session.getCluster()).thenReturn(cluster);
+ PreparedStatement preparedStatement =
Mockito.mock(PreparedStatement.class);
+
when(preparedStatement.bind()).thenReturn(Mockito.mock(BoundStatement.class));
+ when(session.prepare(any(String.class))).thenReturn(preparedStatement);
+
+ Row row = Mockito.mock(Row.class);
+ when(row.getString("name")).thenReturn("concurrent_reads");
+ when(row.getString("value")).thenReturn("16");
+ ResultSet resultSet = Mockito.mock(ResultSet.class);
+ when(resultSet.all()).thenReturn(List.of(row));
+ when(resultSet.one()).thenReturn(row);
+ when(session.execute(argThat((Statement s) ->
+ (s instanceof SimpleStatement) && "SELECT name, value FROM
system_views.settings".equals(
+ ((SimpleStatement) s).getQueryString()
+ )
+ )))
+ .thenReturn(resultSet)
+ .thenThrow(NoHostAvailableException.class);
+
+ CQLSessionProvider cqlSessionProvider =
Mockito.mock(CQLSessionProvider.class);
+ when(cqlSessionProvider.get()).thenReturn(session);
+ when(cqlSessionProvider.getIfConnected()).thenReturn(session);
+ return cqlSessionProvider;
+ }
+
+ private static @NotNull Vertx getMockVertx()
+ {
+ Vertx mockVertx = Mockito.mock(Vertx.class);
+ EventBus mockEventBus = Mockito.mock(EventBus.class);
+ when(mockVertx.eventBus()).thenReturn(mockEventBus);
+ return mockVertx;
+ }
+
+ @Test
+ public void
nativeProtocolHealthCheckWhenHealthCheckSucceedsShouldSetNodeSettingsFromCql()
+ {
+ cassandraAdapterDelegate.nativeProtocolHealthCheck();
+ Map<String, String> actual = cassandraAdapterDelegate.v2NodeSettings();
+ Map<String, String> expected = Map.of("concurrent_reads", "16");
+ Assertions.assertEquals(expected, actual);
+ }
+
+ @Test
+ public void
v2NodeSettingsWhenHealthCheckHasNotRanShouldThrowCassandraUnavailableException()
+ {
+ Assertions.assertThrows(CassandraUnavailableException.class, () ->
cassandraAdapterDelegate.v2NodeSettings());
+ }
+
+ @Test
+ public void
cqlNodeSettingsWhenHealthCheckSucceedsThenFailsShouldUnsetV2NodeSettings()
+ {
+ cassandraAdapterDelegate.nativeProtocolHealthCheck();
+ Map<String, String> actual = cassandraAdapterDelegate.v2NodeSettings();
+ Map<String, String> expected = Map.of("concurrent_reads", "16");
+ Assertions.assertEquals(expected, actual);
+ cassandraAdapterDelegate.v2NodeSettings();
+ cassandraAdapterDelegate.nativeProtocolHealthCheck();
+ Assertions.assertFalse(cassandraAdapterDelegate.isNativeUp());
+ Assertions.assertThrows(CassandraUnavailableException.class, () ->
cassandraAdapterDelegate.v2NodeSettings());
+ }
+}
diff --git
a/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java
b/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java
index 9ee8e5c8..8e21606f 100644
---
a/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java
+++
b/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java
@@ -207,6 +207,8 @@ public class SidecarSchemaTest
"SELECT name, value FROM system_views.settings WHERE name IN ?",
+ "SELECT name, value FROM system_views.settings",
+
"SELECT * FROM system_views.clients",
"SELECT username, COUNT(*) AS connection_count FROM
system_views.clients"
diff --git
a/server/src/test/java/org/apache/cassandra/sidecar/handlers/v2/cassandra/V2NodeSettingsHandlerTest.java
b/server/src/test/java/org/apache/cassandra/sidecar/handlers/v2/cassandra/V2NodeSettingsHandlerTest.java
new file mode 100644
index 00000000..cdc7667a
--- /dev/null
+++
b/server/src/test/java/org/apache/cassandra/sidecar/handlers/v2/cassandra/V2NodeSettingsHandlerTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.handlers.v2.cassandra;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.jupiter.api.Test;
+
+import io.vertx.core.MultiMap;
+import io.vertx.core.Vertx;
+import io.vertx.core.WorkerExecutor;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.cassandra.sidecar.cluster.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.common.response.v2.V2NodeSettings;
+import
org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.config.ServiceConfiguration;
+import org.apache.cassandra.sidecar.config.WorkerPoolConfiguration;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+import org.mockito.Mockito;
+
+/**
+ * The V2NodeSettingsHandlerTest is responsible for verifying that we are able
to properly construct a V2NodeSettings
+ * response in the V2NodeSettings#handle function.
+ */
+public class V2NodeSettingsHandlerTest
+{
+ @Test
+ public void handleShouldWriteNodeSettingsJson()
+ {
+ CassandraAdapterDelegate adapter = buildMockAdapterDelegate();
+ InstanceMetadataFetcher metadataFetcher =
Mockito.mock(InstanceMetadataFetcher.class);
+
Mockito.when(metadataFetcher.delegate(Mockito.any())).thenReturn(adapter);
+ ExecutorPools executorPools = buildMockExecutorPools();
+ V2NodeSettingsHandler v2NodeSettingsHandler = new
V2NodeSettingsHandler(metadataFetcher, executorPools);
+ RoutingContext context = buildMockContext();
+ v2NodeSettingsHandler.handle(context);
+ V2NodeSettings v2NodeSettings = new
V2NodeSettings(Map.of("concurrent_reads", "16"));
+ Mockito.verify(context, Mockito.times(1)).json(v2NodeSettings);
+ }
+
+
+
+ private CassandraAdapterDelegate buildMockAdapterDelegate()
+ {
+ CassandraAdapterDelegate cassandraAdapterDelegate =
Mockito.mock(CassandraAdapterDelegate.class);
+
Mockito.when(cassandraAdapterDelegate.v2NodeSettings()).thenReturn(Map.of("concurrent_reads",
"16"));
+ return cassandraAdapterDelegate;
+ }
+
+ private ExecutorPools buildMockExecutorPools()
+ {
+ ServiceConfiguration sidecarConfig =
Mockito.mock(ServiceConfiguration.class);
+ WorkerPoolConfiguration workerPoolConfiguration =
Mockito.mock(WorkerPoolConfiguration.class);
+
Mockito.when(workerPoolConfiguration.workerPoolName()).thenReturn("name");
+ Mockito.when(workerPoolConfiguration.workerPoolSize()).thenReturn(1);
+ Mockito.when(workerPoolConfiguration.workerMaxExecutionTime())
+ .thenReturn(new MillisecondBoundConfiguration(500,
TimeUnit.MILLISECONDS));
+
Mockito.when(sidecarConfig.serverWorkerPoolConfiguration()).thenReturn(workerPoolConfiguration);
+
Mockito.when(sidecarConfig.serverInternalWorkerPoolConfiguration()).thenReturn(workerPoolConfiguration);
+ Vertx vertx = Mockito.mock(Vertx.class);
+ Mockito.when(vertx.createSharedWorkerExecutor(Mockito.any()))
+ .thenReturn(Mockito.mock(WorkerExecutor.class));
+ return new ExecutorPools(vertx, sidecarConfig);
+ }
+
+ private RoutingContext buildMockContext()
+ {
+ RoutingContext context = Mockito.mock(RoutingContext.class);
+ HttpServerRequest request = Mockito.mock(HttpServerRequest.class);
+
Mockito.when(request.remoteAddress()).thenReturn(Mockito.mock(SocketAddress.class));
+ Mockito.when(request.host()).thenReturn("127.0.0.1");
+
Mockito.when(request.params()).thenReturn(Mockito.mock(MultiMap.class));
+ Mockito.when(context.request()).thenReturn(request);
+ return context;
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]