CASSANDRASC-84: Expose additional node settings

Sidecar exposes settings from the Cassandra node via the node settings API endpoint. The information exposed is
limited, and we need to start exposing additional information from the `system.local` table, for example
`datacenter` information, owned token ranges, and the local address and port for the native protocol. This
information can be consumed by Sidecar itself, as well as the Cassandra Analytics library.

In this commit, we expose additional settings for the node.

Patch by Francisco Guerrero; Reviewed by Yifan Cai for CASSANDRASC-84
diff --git a/CHANGES.txt b/CHANGES.txt
index f7b0572..8e38818 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 1.0.0
 -----
+ * Expose additional node settings (CASSANDRASC-84)
  * 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)
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
index a730bfe..a453740 100644
--- a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraAdapter.java
@@ -34,6 +34,13 @@
 import org.apache.cassandra.sidecar.common.dns.DnsResolver;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.cassandra.sidecar.common.NodeSettings.DATA_CENTER_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.PARTITIONER_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.RELEASE_VERSION_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.RPC_ADDRESS_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.RPC_PORT_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.TOKENS_COLUMN_NAME;
+
 /**
  * A {@link ICassandraAdapter} implementation for Cassandra 4.0 and later
  */
@@ -96,12 +103,25 @@
             return null;
         }
 
-        Row oneResult = activeSession.execute("select release_version, partitioner from system.local")
+        Row oneResult = activeSession.execute("SELECT "
+                                              + RELEASE_VERSION_COLUMN_NAME + ", "
+                                              + PARTITIONER_COLUMN_NAME + ", "
+                                              + DATA_CENTER_COLUMN_NAME + ", "
+                                              + RPC_ADDRESS_COLUMN_NAME + ", "
+                                              + RPC_PORT_COLUMN_NAME + ", "
+                                              + TOKENS_COLUMN_NAME
+                                              + " FROM system.local")
                                      .one();
 
-        return new NodeSettings(oneResult.getString("release_version"),
-                                oneResult.getString("partitioner"),
-                                sidecarVersion);
+        return NodeSettings.builder()
+                           .releaseVersion(oneResult.getString(RELEASE_VERSION_COLUMN_NAME))
+                           .partitioner(oneResult.getString(PARTITIONER_COLUMN_NAME))
+                           .sidecarVersion(sidecarVersion)
+                           .datacenter(oneResult.getString(DATA_CENTER_COLUMN_NAME))
+                           .tokens(oneResult.getSet(TOKENS_COLUMN_NAME, String.class))
+                           .rpcAddress(oneResult.getInet(RPC_ADDRESS_COLUMN_NAME))
+                           .rpcPort(oneResult.getInt(RPC_PORT_COLUMN_NAME))
+                           .build();
     }
 
     /**
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 8833403..61dae74 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
@@ -18,9 +18,11 @@
 
 package org.apache.cassandra.sidecar.common;
 
+import java.net.InetAddress;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 
@@ -31,38 +33,50 @@
 {
     private static final String VERSION = "version";
 
+    public static final String RELEASE_VERSION_COLUMN_NAME = "release_version";
+    public static final String PARTITIONER_COLUMN_NAME = "partitioner";
+    public static final String DATA_CENTER_COLUMN_NAME = "data_center";
+    public static final String RPC_ADDRESS_COLUMN_NAME = "rpc_address";
+    public static final String RPC_PORT_COLUMN_NAME = "rpc_port";
+    public static final String TOKENS_COLUMN_NAME = "tokens";
+
+    @JsonProperty("releaseVersion")
     private final String releaseVersion;
+    @JsonProperty("partitioner")
     private final String partitioner;
+    @JsonProperty("datacenter")
+    private final String datacenter;
+    @JsonProperty("rpcAddress")
+    private final InetAddress rpcAddress;
+    @JsonProperty("rpcPort")
+    private final int rpcPort;
+    @JsonProperty("tokens")
+    private final Set<String> tokens;
+    @JsonProperty("sidecar")
     private final Map<String, String> sidecar;
 
     /**
-     * Constructs a new {@link NodeSettings} object with the Cassandra node's release version,
-     * partitioner, and Sidecar version information
-     *
-     * @param releaseVersion the release version of the Cassandra node
-     * @param partitioner    the partitioner used by the Cassandra node
-     * @param sidecarVersion the version of the Sidecar on the Cassandra node
+     * Constructs a new {@link NodeSettings}.
      */
-    public NodeSettings(String releaseVersion, String partitioner, String sidecarVersion)
+    public NodeSettings()
     {
-        this(releaseVersion, partitioner, Collections.singletonMap(VERSION, sidecarVersion));
+        this(builder());
     }
 
     /**
-     * Constructs a new {@link NodeSettings} object with the Cassandra node's release version,
-     * partitioner, and Sidecar settings information
+     * Constructs a new {@link NodeSettings} from the configured {@link Builder}.
      *
-     * @param releaseVersion the release version of the Cassandra node
-     * @param partitioner    the partitioner used by the Cassandra node
-     * @param sidecar        the settings of the Sidecar on the Cassandra node, including its version
+     * @param builder the builder used to create this object
      */
-    public NodeSettings(@JsonProperty("releaseVersion") String releaseVersion,
-                        @JsonProperty("partitioner") String partitioner,
-                        @JsonProperty("sidecar") Map<String, String> sidecar)
+    protected NodeSettings(Builder builder)
     {
-        this.releaseVersion = releaseVersion;
-        this.partitioner = partitioner;
-        this.sidecar = sidecar;
+        releaseVersion = builder.releaseVersion;
+        partitioner = builder.partitioner;
+        datacenter = builder.datacenter;
+        rpcAddress = builder.rpcAddress;
+        rpcPort = builder.rpcPort;
+        tokens = builder.tokens;
+        sidecar = builder.sidecar;
     }
 
     @JsonProperty("releaseVersion")
@@ -88,6 +102,30 @@
         return sidecar != null ? sidecar.get(VERSION) : "unknown";
     }
 
+    @JsonProperty("datacenter")
+    public String datacenter()
+    {
+        return datacenter;
+    }
+
+    @JsonProperty("rpcAddress")
+    public InetAddress rpcAddress()
+    {
+        return rpcAddress;
+    }
+
+    @JsonProperty("rpcPort")
+    public int rpcPort()
+    {
+        return rpcPort;
+    }
+
+    @JsonProperty("tokens")
+    public Set<String> tokens()
+    {
+        return tokens;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -105,7 +143,12 @@
         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.sidecar, that.sidecar)
+               && Objects.equals(this.datacenter, that.datacenter)
+               && Objects.equals(this.rpcAddress, that.rpcAddress)
+               && Objects.equals(this.rpcPort, that.rpcPort)
+               && Objects.equals(this.tokens, that.tokens)
+        ;
     }
 
     /**
@@ -114,6 +157,147 @@
     @Override
     public int hashCode()
     {
-        return Objects.hash(releaseVersion, partitioner, sidecar);
+        return Objects.hash(releaseVersion, partitioner, sidecar, datacenter, rpcAddress, rpcPort, tokens);
+    }
+
+    /**
+     * @return a new NodeSettings builder
+     */
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    /**
+     * {@code NodeSettings} builder static inner class.
+     */
+    public static final class Builder implements DataObjectBuilder<Builder, NodeSettings>
+    {
+        private String releaseVersion;
+        private String partitioner;
+        private String datacenter;
+        private InetAddress rpcAddress;
+        private int rpcPort;
+        private Set<String> tokens;
+        private Map<String, String> sidecar;
+
+        private Builder()
+        {
+        }
+
+        @Override
+        public Builder self()
+        {
+            return this;
+        }
+
+        /**
+         * Sets the {@code releaseVersion} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param releaseVersion the {@code releaseVersion} to set
+         * @return a reference to this Builder
+         */
+        public Builder releaseVersion(String releaseVersion)
+        {
+            return update(b -> b.releaseVersion = releaseVersion);
+        }
+
+        /**
+         * Sets the {@code partitioner} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param partitioner the {@code partitioner} to set
+         * @return a reference to this Builder
+         */
+        public Builder partitioner(String partitioner)
+        {
+            return update(b -> b.partitioner = partitioner);
+        }
+
+        /**
+         * Sets the {@code datacenter} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param datacenter the {@code datacenter} to set
+         * @return a reference to this Builder
+         */
+        public Builder datacenter(String datacenter)
+        {
+            return update(b -> b.datacenter = datacenter);
+        }
+
+        /**
+         * Sets the {@code rpcAddress} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param rpcAddress the {@code rpcAddress} to set
+         * @return a reference to this Builder
+         */
+        public Builder rpcAddress(InetAddress rpcAddress)
+        {
+            return update(b -> b.rpcAddress = rpcAddress);
+        }
+
+        /**
+         * Sets the {@code rpcPort} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param rpcPort the {@code rpcPort} to set
+         * @return a reference to this Builder
+         */
+        public Builder rpcPort(int rpcPort)
+        {
+            return update(b -> b.rpcPort = rpcPort);
+        }
+
+        /**
+         * Sets the {@code tokens} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param tokens the {@code tokens} to set
+         * @return a reference to this Builder
+         */
+        public Builder tokens(Set<String> tokens)
+        {
+            return update(b -> b.tokens = tokens);
+        }
+
+        /**
+         * Sets the {@code sidecar} and returns a reference to this Builder enabling method chaining.
+         *
+         * @param sidecar the {@code sidecar} to set
+         * @return a reference to this Builder
+         */
+        public Builder sidecar(Map<String, String> sidecar)
+        {
+            return update(b -> b.sidecar = sidecar);
+        }
+
+        /**
+         * Sets the {@code sidecarVersion} in the {@code sidecar} map and returns a reference to this Builder
+         * enabling method chaining.
+         *
+         * @param sidecarVersion the {@code sidecarVersion} to set
+         * @return a reference to this Builder
+         */
+        public Builder sidecarVersion(String sidecarVersion)
+        {
+            return update(b -> {
+                if (b.sidecar != null)
+                {
+                    b.sidecar.put(VERSION, sidecarVersion);
+                }
+                else
+                {
+                    b.sidecar = Collections.singletonMap(VERSION, sidecarVersion);
+                }
+            });
+        }
+
+        /**
+         * Returns a {@code NodeSettings} built from the parameters previously set.
+         *
+         * @return a {@code NodeSettings} built with parameters of this {@code NodeSettings.Builder}
+         */
+        @Override
+        public NodeSettings build()
+        {
+            return new NodeSettings(this);
+        }
     }
 }
diff --git a/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java b/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
index 7466a8d..115f188 100644
--- a/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
+++ b/src/main/java/org/apache/cassandra/sidecar/cluster/CassandraAdapterDelegate.java
@@ -46,6 +46,12 @@
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.cassandra.sidecar.common.NodeSettings.DATA_CENTER_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.PARTITIONER_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.RELEASE_VERSION_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.RPC_ADDRESS_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.RPC_PORT_COLUMN_NAME;
+import static org.apache.cassandra.sidecar.common.NodeSettings.TOKENS_COLUMN_NAME;
 import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CASSANDRA_CQL_DISCONNECTED;
 import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CASSANDRA_CQL_READY;
 
@@ -166,14 +172,28 @@
 
         try
         {
-            Row oneResult = activeSession.execute("select release_version, partitioner from system.local")
+            Row oneResult = activeSession.execute("SELECT "
+                                                  + RELEASE_VERSION_COLUMN_NAME + ", "
+                                                  + PARTITIONER_COLUMN_NAME + ", "
+                                                  + DATA_CENTER_COLUMN_NAME + ", "
+                                                  + RPC_ADDRESS_COLUMN_NAME + ", "
+                                                  + RPC_PORT_COLUMN_NAME + ", "
+                                                  + TOKENS_COLUMN_NAME
+                                                  + " FROM system.local")
                                          .one();
 
             // Note that within the scope of this method, we should keep on using the local releaseVersion
-            String releaseVersion = oneResult.getString("release_version");
-            NodeSettings newNodeSettings = new NodeSettings(releaseVersion,
-                                                            oneResult.getString("partitioner"),
-                                                            sidecarVersion);
+            String releaseVersion = oneResult.getString(RELEASE_VERSION_COLUMN_NAME);
+            NodeSettings newNodeSettings = NodeSettings.builder()
+                                                       .releaseVersion(releaseVersion)
+                                                       .partitioner(oneResult.getString(PARTITIONER_COLUMN_NAME))
+                                                       .sidecarVersion(sidecarVersion)
+                                                       .datacenter(oneResult.getString(DATA_CENTER_COLUMN_NAME))
+                                                       .tokens(oneResult.getSet(TOKENS_COLUMN_NAME, String.class))
+                                                       .rpcAddress(oneResult.getInet(RPC_ADDRESS_COLUMN_NAME))
+                                                       .rpcPort(oneResult.getInt(RPC_PORT_COLUMN_NAME))
+                                                       .build();
+
             if (!newNodeSettings.equals(nodeSettings))
             {
                 // Update the nodeSettings cache
diff --git a/src/test/java/org/apache/cassandra/sidecar/TestModule.java b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
index d78d11c..d639944 100644
--- a/src/test/java/org/apache/cassandra/sidecar/TestModule.java
+++ b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.sidecar;
 
+import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -121,8 +122,15 @@
         CassandraAdapterDelegate delegate = mock(CassandraAdapterDelegate.class);
         if (isUp)
         {
-            when(delegate.nodeSettings()).thenReturn(new NodeSettings(
-            "testVersion", "testPartitioner", Collections.singletonMap("version", "testSidecar")));
+            when(delegate.nodeSettings()).thenReturn(NodeSettings.builder()
+                                                                 .releaseVersion("testVersion")
+                                                                 .partitioner("testPartitioner")
+                                                                 .sidecarVersion("testSidecar")
+                                                                 .datacenter("testDC")
+                                                                 .rpcAddress(InetAddress.getLoopbackAddress())
+                                                                 .rpcPort(6475)
+                                                                 .tokens(Collections.singleton("testToken"))
+                                                                 .build());
         }
         when(delegate.isUp()).thenReturn(isUp);
         when(instanceMeta.delegate()).thenReturn(delegate);
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/cassandra/NodeSettingsHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/cassandra/NodeSettingsHandlerTest.java
index 2d9f0af..b9f7cf8 100644
--- a/src/test/java/org/apache/cassandra/sidecar/routes/cassandra/NodeSettingsHandlerTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/cassandra/NodeSettingsHandlerTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.sidecar.routes.cassandra;
 
+import java.net.InetAddress;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -31,8 +32,11 @@
 import com.google.inject.Injector;
 import com.google.inject.util.Modules;
 import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.AsyncResult;
 import io.vertx.core.Vertx;
+import io.vertx.core.buffer.Buffer;
 import io.vertx.core.json.JsonObject;
+import io.vertx.ext.web.client.HttpResponse;
 import io.vertx.ext.web.client.WebClient;
 import io.vertx.ext.web.codec.BodyCodec;
 import io.vertx.junit5.VertxExtension;
@@ -81,39 +85,33 @@
     }
 
     @Test
-    public void validRequestWithoutInstanceId(VertxTestContext context)
+    void validRequestWithoutInstanceId(VertxTestContext context)
     {
         WebClient client = WebClient.create(vertx);
         client.get(server.actualPort(), "localhost", NODE_SETTINGS_ROUTE)
               .as(BodyCodec.buffer())
               .send(resp -> {
                   assertThat(resp.result().statusCode()).isEqualTo(HttpResponseStatus.OK.code());
-                  NodeSettings status = resp.result().bodyAsJson(NodeSettings.class);
-                  assertThat(status.partitioner()).isEqualTo("testPartitioner");
-                  assertThat(status.releaseVersion()).isEqualTo("testVersion");
-                  assertThat(status.sidecarVersion()).isEqualTo("testSidecar");
+                  validateNodeSettings(resp);
                   context.completeNow();
               });
     }
 
     @Test
-    public void validRequestWithInstanceId(VertxTestContext context)
+    void validRequestWithInstanceId(VertxTestContext context)
     {
         WebClient client = WebClient.create(vertx);
         client.get(server.actualPort(), "localhost", String.format(URI_WITH_INSTANCE_ID, "1"))
               .as(BodyCodec.buffer())
               .send(resp -> {
                   assertThat(resp.result().statusCode()).isEqualTo(HttpResponseStatus.OK.code());
-                  NodeSettings status = resp.result().bodyAsJson(NodeSettings.class);
-                  assertThat(status.partitioner()).isEqualTo("testPartitioner");
-                  assertThat(status.releaseVersion()).isEqualTo("testVersion");
-                  assertThat(status.sidecarVersion()).isEqualTo("testSidecar");
+                  validateNodeSettings(resp);
                   context.completeNow();
               });
     }
 
     @Test
-    public void validRequestWithInvalidInstanceId(VertxTestContext context)
+    void validRequestWithInvalidInstanceId(VertxTestContext context)
     {
         WebClient client = WebClient.create(vertx);
         client.get(server.actualPort(), "localhost", String.format(URI_WITH_INSTANCE_ID, "10"))
@@ -126,4 +124,18 @@
                   context.completeNow();
               });
     }
+
+    static void validateNodeSettings(AsyncResult<HttpResponse<Buffer>> resp)
+    {
+        NodeSettings status = resp.result().bodyAsJson(NodeSettings.class);
+        assertThat(status.partitioner()).isEqualTo("testPartitioner");
+        assertThat(status.releaseVersion()).isEqualTo("testVersion");
+        assertThat(status.sidecarVersion()).isEqualTo("testSidecar");
+        assertThat(status.datacenter()).isEqualTo("testDC");
+        assertThat(status.rpcAddress()).isEqualTo(InetAddress.getLoopbackAddress());
+        assertThat(status.rpcPort()).isEqualTo(6475);
+        assertThat(status.tokens()).isNotNull()
+                                   .isNotEmpty()
+                                   .containsExactly("testToken");
+    }
 }