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]


Reply via email to