You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "arjunashok (via GitHub)" <gi...@apache.org> on 2023/06/28 22:29:47 UTC

[GitHub] [cassandra-sidecar] arjunashok opened a new pull request, #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

arjunashok opened a new pull request, #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58

   …n-ranges
   
   This commit introduces a new sidecar endpoint (`token-range-replicas`) which exposes cluster topology information by keyspace, namely token ranges and endpoint mappings (including natural and pending ranges). 
   
   The response consists of token ranges mapped to read and write replica-sets grouped by datacenter .  Write replica-sets will include mappings for pending ranges, mapping to nodes in transient states such as joining or leaving.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313029686


##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());

Review Comment:
   NIT: The size comparison isn't necessary given `containsExactlyElementsOf`"verifies that actual contains exactly the elements of the given iterable and nothing else in the same order."



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1306204877


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)

Review Comment:
   The `protected` methods are not for extension, since there is no subclass. Please not use `protected`.



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());

Review Comment:
   nit: add some indentation to make it easier to read
   
   ```suggestion
           List<TokenRangeReplicas> replicas = Stream.concat(naturalReplicaMappings.entrySet().stream(), 
                                                             pendingRangeMappings.entrySet().stream())
                                                     .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
                                                         new BigInteger(entry.getKey().get(0)),
                                                         new BigInteger(entry.getKey().get(1)),
                                                         partitioner,
                                                         new HashSet<>(entry.getValue())))
                                                     .flatMap(Collection::stream)
                                                     .collect(Collectors.toList());
   ```



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)

Review Comment:
   nit: readability
   ```suggestion
           return replicasByTokenRange.entrySet().stream()
                                      .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
                                          new BigInteger(entry.getKey().get(0)),
                                          new BigInteger(entry.getKey().get(1)),
                                          partitioner,
                                          new HashSet<>(entry.getValue())))
                                      .flatMap(Collection::stream)
   ```



##########
adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java:
##########
@@ -0,0 +1,847 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    private boolean hasOverlaps(List<TokenRangeReplicas> rangeList)
+    {
+        Collections.sort(rangeList);
+        for (int c = 0, i = 1; i < rangeList.size(); i++)
+        {
+            if (rangeList.get(c++).end().compareTo(rangeList.get(i).start()) > 0) return true;
+        }
+        return false;
+    }
+
+    private boolean checkContains(List<TokenRangeReplicas> resultList, TokenRangeReplicas expected)
+    {
+        return resultList.stream()
+                         .map(TokenRangeReplicas::toString)
+                         .anyMatch(r -> r.equals(expected.toString()));
+    }
+
+    // non-overlapping ranges
+    @Test
+    public void simpleTest()
+    {
+        List<TokenRangeReplicas> simpleList = createSimpleTokenRangeReplicaList();
+        LOGGER.info("Input:" + simpleList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(simpleList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+    }
+
+    // TODO: Validate unwrapping separately

Review Comment:
   Not sure if it is addressed by `wrappedMultiOverlapTest` already. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/CQLSessionProvider.java:
##########
@@ -126,8 +126,15 @@ public synchronized Session localCql()
         return localSession;
     }
 
-    public synchronized void close()
+    public Session close()
     {
+        Session localSession;
+        synchronized (this)
+        {
+            localSession = this.localSession;
+            this.localSession = null;
+        }
+

Review Comment:
   👍 



##########
src/test/integration/org/apache/cassandra/sidecar/routes/MultiDcTokenSupplier.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.distributed.api.TokenSupplier;
+
+/**
+ * Static factory holder that provides a multi-DC token supplier
+ */
+public class MultiDcTokenSupplier
+{
+
+    static TokenSupplier evenlyDistributedTokens(int numNodes, int numDcs, int numTokens)

Review Comment:
   I guess you mean this
   ```suggestion
       static TokenSupplier evenlyDistributedTokens(int numNodesPerDc, int numDcs, int numTokensPerNode)
   ```
   
   Then, what does the result of `numTokensPerNode * numDcs` mean? 



##########
src/test/integration/org/apache/cassandra/sidecar/routes/MultiDcTokenSupplier.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.distributed.api.TokenSupplier;
+
+/**
+ * Static factory holder that provides a multi-DC token supplier
+ */
+public class MultiDcTokenSupplier
+{
+
+    static TokenSupplier evenlyDistributedTokens(int numNodes, int numDcs, int numTokens)
+    {
+        long totalTokens = (long) numNodes * numDcs * numTokens;
+        BigInteger increment = BigInteger.valueOf((Long.MAX_VALUE / totalTokens) * 4);

Review Comment:
   Where is `4` coming from? 



##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,795 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.description.type.TypeDescription;
+import net.bytebuddy.dynamic.ClassFileLocator;
+import net.bytebuddy.dynamic.TypeResolutionStrategy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.pool.TypePool;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import org.apache.cassandra.utils.Shared;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static net.bytebuddy.matcher.ElementMatchers.takesArguments;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.apache.cassandra.sidecar.routes.TokenRangeIntegrationMovingTest.MOVING_NODE_IDX;
+import static org.apache.cassandra.sidecar.routes.TokenRangeIntegrationMovingTest.MULTIDC_MOVING_NODE_IDX;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());
+        assertThat(writeRanges).containsExactlyElementsOf(expectedRanges);
+
+        //Sorted and Overlap check
+        validateOrderAndOverlaps(writeRanges);
+        validateOrderAndOverlaps(readRanges);
+    }
+
+    private void validateOrderAndOverlaps(List<Range<BigInteger>> ranges)
+    {
+        for (int r = 0; r < ranges.size() - 1; r++)
+        {
+            assertThat(ranges.get(r).upperEndpoint()).isLessThan(ranges.get(r + 1).upperEndpoint());
+            assertThat(ranges.get(r).intersection(ranges.get(r + 1)).isEmpty()).isTrue();
+        }
+    }
+
+    protected void validateNodeStates(TokenRangeReplicasResponse mappingResponse,
+                                      Set<String> dcReplication,
+                                      Function<Integer, String> statusFunction)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int expectedReplicas = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * dcReplication.size();
+
+        AbstractCassandraTestContext cassandraTestContext = sidecarTestContext.cassandraTestContext();
+        assertThat(mappingResponse.replicaState().size()).isEqualTo(expectedReplicas);
+        for (int i = 1; i <= cassandraTestContext.cluster().size(); i++)
+        {
+            IInstanceConfig config = cassandraTestContext.cluster().get(i).config();
+
+            if (dcReplication.contains(config.localDatacenter()))
+            {
+                String ipAndPort = config.broadcastAddress().getAddress().getHostAddress() + ":"
+                                   + config.broadcastAddress().getPort();
+
+                String expectedStatus = statusFunction.apply(i);
+                assertThat(mappingResponse.replicaState().get(ipAndPort)).isEqualTo(expectedStatus);
+            }
+        }
+    }
+
+    protected UpgradeableCluster getMultiDCCluster(int numNodes,
+                                                   int numDcs,
+                                                   BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws IOException
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier =
+        MultiDcTokenSupplier.evenlyDistributedTokens(numNodes,
+                                                     numDcs,
+                                                     1);
+
+        int totalNodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return cassandraTestContext.configureAndStartCluster(
+        builder -> {
+            builder.withInstanceInitializer(initializer);
+            builder.withTokenSupplier(mdcTokenSupplier);
+            builder.withNodeIdTopology(networkTopology(totalNodeCount,
+                                                       (nodeId) -> nodeId % 2 != 0 ?
+                                                                   dcAndRack("datacenter1", "rack1") :
+                                                                   dcAndRack("datacenter2", "rack2")));
+        });
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return generateExpectedRanges(nodeCount);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(int nodeCount)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = (annotation.numDcs() > 1) ?
+                                      MultiDcTokenSupplier.evenlyDistributedTokens(
+                                      annotation.nodesPerDc() + annotation.newNodesPerDc(),
+                                      annotation.numDcs(),
+                                      1) :
+                                      TokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc() +
+                                                                            annotation.newNodesPerDc(),
+                                                                            1);
+
+        List<Range<BigInteger>> expectedRanges = new ArrayList<>();
+        BigInteger startToken = Partitioner.Murmur3.minToken;
+        BigInteger endToken = Partitioner.Murmur3.maxToken;
+        int node = 1;
+        BigInteger prevToken = new BigInteger(tokenSupplier.tokens(node++).stream().findFirst().get());
+        Range<BigInteger> firstRange = Range.openClosed(startToken, prevToken);
+        expectedRanges.add(firstRange);
+        while (node <= nodeCount)
+        {
+            BigInteger currentToken = new BigInteger(tokenSupplier.tokens(node).stream().findFirst().get());
+            expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            prevToken = currentToken;
+            node++;
+        }
+        expectedRanges.add(Range.openClosed(prevToken, endToken));
+        return expectedRanges;
+    }
+
+    protected Set<String> instancesFromReplicaSet(List<TokenRangeReplicasResponse.ReplicaInfo> replicas)
+    {
+        return replicas.stream()
+                       .map(r -> r.replicasByDatacenter().values())
+                       .flatMap(Collection::stream)
+                       .flatMap(Collection::stream)
+                       .collect(Collectors.toSet());

Review Comment:
   nit: one less line if you do this 
   ```suggestion
           return replicas.stream()
                          .flatMap(r -> r.replicasByDatacenter().values().stream())
                          .flatMap(Collection::stream)
                          .collect(Collectors.toSet());
   ```



##########
src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.net.UnknownHostException;
+
+import org.apache.commons.lang3.StringUtils;
+
+import com.datastax.driver.core.Metadata;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.handler.HttpException;
+import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasRequest;
+import org.apache.cassandra.sidecar.common.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static org.apache.cassandra.sidecar.utils.HttpExceptions.cassandraServiceUnavailable;
+
+/**
+ * Handler which provides token range to read and write replica mapping
+ *
+ * <p>This handler provides token range replicas along with the state of the replicas. For the purpose
+ * of identifying the state of a newly joining node to replace a dead node from a newly joining node,
+ * a new state 'Replacing' has been added.
+ * It is represented by
+ * {@code org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicaProvider.StateWithReplacement}
+ */
+@Singleton
+public class TokenRangeReplicaMapHandler extends AbstractHandler<TokenRangeReplicasRequest>
+{
+
+    @Inject
+    public TokenRangeReplicaMapHandler(InstanceMetadataFetcher metadataFetcher,
+                                       CassandraInputValidator validator,
+                                       ExecutorPools executorPools)
+    {
+        super(metadataFetcher, executorPools, validator);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void handleInternal(RoutingContext context,
+                               HttpServerRequest httpRequest,
+                               String host,
+                               SocketAddress remoteAddress,
+                               TokenRangeReplicasRequest request)
+    {
+        CassandraAdapterDelegate delegate = metadataFetcher.delegate(host);
+
+        StorageOperations storageOperations = delegate.storageOperations();
+        Metadata metadata = delegate.metadata();
+        if (storageOperations == null || metadata == null)
+        {
+            context.fail(cassandraServiceUnavailable());
+            return;
+        }
+
+        executorPools.service().executeBlocking(promise -> {
+            try
+            {
+                context.json(storageOperations.tokenRangeReplicas(request.keyspace(), metadata.getPartitioner()));
+            }
+            catch (UnknownHostException e)
+            {
+                processFailure(e, context, host, remoteAddress, request);

Review Comment:
   right. The method in the interface is declared to throw `UnknownHostException`, but the implementation does not. It is wrapped in `RuntimeException`. Can you update the interface method to remove the throws? 
   The `RuntimeException` does not need to be handled separately, it reaches to `onFailure`, which calls `processFailure`, if throws. 



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);

Review Comment:
   Not an AI, but maybe refactor the code in a follow-up, so that `jmxClient` is not available in the scope to prevent calling `jmxClient.proxy` directly.



##########
build.gradle:
##########
@@ -326,6 +326,8 @@ tasks.register("integrationTest", Test) {
     useJUnitPlatform() {
         includeTags "integrationTest"
     }
+// Uncomment below to run unit tests in parallel
+//    maxParallelForks = Runtime.runtime.availableProcessors() * 2

Review Comment:
   Just curious, why it is commented out?



##########
common/build.gradle:
##########
@@ -41,6 +41,8 @@ repositories {
 
 test {
     useJUnitPlatform()
+// Uncomment below to run unit tests in parallel
+//    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1

Review Comment:
   Another curiosity, why the `maxParallelForks` calculations are different in different build files?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";
+        private final Set<String> joiningNodes;
+        private final GossipInfoResponse gossipInfo;
+
+        StateWithReplacement(List<String> joiningNodes, List<String> leavingNodes, List<String> movingNodes,
+                             GossipInfoResponse gossipInfo)
+        {
+            super(joiningNodes, leavingNodes, movingNodes);
+            this.joiningNodes = new HashSet<>(joiningNodes);
+            this.gossipInfo = gossipInfo;
+        }
+
+        /**
+         * This method returns state of a node and accounts for a new 'Replacing' state if the node is
+         * replacing a dead node. For returning this state, the method checks status of the node in gossip
+         * information.
+         *
+         * @param endpoint node information represented usually in form of 'ip:port'
+         * @return Node status
+         */
+        @Override
+        String of(String endpoint)
+        {
+            if (joiningNodes.contains(endpoint))
+            {
+                GossipInfoResponse.GossipInfo gossipInfoEntry = gossipInfo.get(endpoint);
+
+                if (gossipInfoEntry != null)
+                {
+                    LOGGER.info("Found gossipInfoEntry={}", gossipInfoEntry);

Review Comment:
   nit: should we only print the endpoint and status? Those 2 values should be enough to reason about the runtime here.



##########
common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java:
##########
@@ -259,12 +259,16 @@ private Map<String, Object> buildJmxEnv()
     }
 
     @Override
-    public synchronized void close() throws IOException
+    public void close() throws IOException
     {
-        JMXConnector connector = jmxConnector;
-        if (connector != null)
+        JMXConnector connector;
+        synchronized (this)
         {
+            connector = jmxConnector;
             jmxConnector = null;

Review Comment:
   Reset `connected` to false too?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";

Review Comment:
   How about create an enum for the state? It helps in 2 ways, strong typed and comprehensive matching. 



##########
common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java:
##########
@@ -259,12 +259,16 @@ private Map<String, Object> buildJmxEnv()
     }
 
     @Override
-    public synchronized void close() throws IOException
+    public void close() throws IOException
     {
-        JMXConnector connector = jmxConnector;
-        if (connector != null)
+        JMXConnector connector;
+        synchronized (this)

Review Comment:
   Can you make `connect()` (where `jmxConnector` is update) synchronized? It is synchronized in its only callsite. It won't hurt performance when synchronizing `connnect()`, but more error-proof.



##########
common/src/main/java/org/apache/cassandra/sidecar/common/CQLSessionProvider.java:
##########
@@ -126,8 +126,15 @@ public synchronized Session localCql()
         return localSession;
     }
 
-    public synchronized void close()
+    public Session close()
     {
+        Session localSession;
+        synchronized (this)
+        {
+            localSession = this.localSession;
+            this.localSession = null;
+        }
+

Review Comment:
   👍 



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";
+        private final Set<String> joiningNodes;

Review Comment:
   Why does it declare another copy of `joiningNodes`? It is available from the base class (but need to make it protected or add getter).



##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,795 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.description.type.TypeDescription;
+import net.bytebuddy.dynamic.ClassFileLocator;
+import net.bytebuddy.dynamic.TypeResolutionStrategy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.pool.TypePool;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import org.apache.cassandra.utils.Shared;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static net.bytebuddy.matcher.ElementMatchers.takesArguments;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.apache.cassandra.sidecar.routes.TokenRangeIntegrationMovingTest.MOVING_NODE_IDX;
+import static org.apache.cassandra.sidecar.routes.TokenRangeIntegrationMovingTest.MULTIDC_MOVING_NODE_IDX;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());
+        assertThat(writeRanges).containsExactlyElementsOf(expectedRanges);
+
+        //Sorted and Overlap check
+        validateOrderAndOverlaps(writeRanges);
+        validateOrderAndOverlaps(readRanges);
+    }
+
+    private void validateOrderAndOverlaps(List<Range<BigInteger>> ranges)
+    {
+        for (int r = 0; r < ranges.size() - 1; r++)
+        {
+            assertThat(ranges.get(r).upperEndpoint()).isLessThan(ranges.get(r + 1).upperEndpoint());
+            assertThat(ranges.get(r).intersection(ranges.get(r + 1)).isEmpty()).isTrue();
+        }
+    }
+
+    protected void validateNodeStates(TokenRangeReplicasResponse mappingResponse,
+                                      Set<String> dcReplication,
+                                      Function<Integer, String> statusFunction)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int expectedReplicas = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * dcReplication.size();
+
+        AbstractCassandraTestContext cassandraTestContext = sidecarTestContext.cassandraTestContext();
+        assertThat(mappingResponse.replicaState().size()).isEqualTo(expectedReplicas);
+        for (int i = 1; i <= cassandraTestContext.cluster().size(); i++)
+        {
+            IInstanceConfig config = cassandraTestContext.cluster().get(i).config();
+
+            if (dcReplication.contains(config.localDatacenter()))
+            {
+                String ipAndPort = config.broadcastAddress().getAddress().getHostAddress() + ":"
+                                   + config.broadcastAddress().getPort();
+
+                String expectedStatus = statusFunction.apply(i);
+                assertThat(mappingResponse.replicaState().get(ipAndPort)).isEqualTo(expectedStatus);
+            }
+        }
+    }
+
+    protected UpgradeableCluster getMultiDCCluster(int numNodes,
+                                                   int numDcs,
+                                                   BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws IOException
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier =
+        MultiDcTokenSupplier.evenlyDistributedTokens(numNodes,
+                                                     numDcs,
+                                                     1);
+
+        int totalNodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return cassandraTestContext.configureAndStartCluster(
+        builder -> {
+            builder.withInstanceInitializer(initializer);
+            builder.withTokenSupplier(mdcTokenSupplier);
+            builder.withNodeIdTopology(networkTopology(totalNodeCount,
+                                                       (nodeId) -> nodeId % 2 != 0 ?
+                                                                   dcAndRack("datacenter1", "rack1") :
+                                                                   dcAndRack("datacenter2", "rack2")));
+        });
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return generateExpectedRanges(nodeCount);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(int nodeCount)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = (annotation.numDcs() > 1) ?
+                                      MultiDcTokenSupplier.evenlyDistributedTokens(
+                                      annotation.nodesPerDc() + annotation.newNodesPerDc(),
+                                      annotation.numDcs(),
+                                      1) :
+                                      TokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc() +
+                                                                            annotation.newNodesPerDc(),
+                                                                            1);
+
+        List<Range<BigInteger>> expectedRanges = new ArrayList<>();
+        BigInteger startToken = Partitioner.Murmur3.minToken;
+        BigInteger endToken = Partitioner.Murmur3.maxToken;
+        int node = 1;
+        BigInteger prevToken = new BigInteger(tokenSupplier.tokens(node++).stream().findFirst().get());
+        Range<BigInteger> firstRange = Range.openClosed(startToken, prevToken);
+        expectedRanges.add(firstRange);
+        while (node <= nodeCount)
+        {
+            BigInteger currentToken = new BigInteger(tokenSupplier.tokens(node).stream().findFirst().get());
+            expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            prevToken = currentToken;
+            node++;
+        }
+        expectedRanges.add(Range.openClosed(prevToken, endToken));
+        return expectedRanges;
+    }
+
+    protected Set<String> instancesFromReplicaSet(List<TokenRangeReplicasResponse.ReplicaInfo> replicas)
+    {
+        return replicas.stream()
+                       .map(r -> r.replicasByDatacenter().values())
+                       .flatMap(Collection::stream)
+                       .flatMap(Collection::stream)
+                       .collect(Collectors.toSet());
+    }
+
+    protected void validateWriteReplicaMappings(List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas,
+                                              Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMapping)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        assertThat(writeReplicas).hasSize(expectedRangeMapping.get("datacenter1").size());
+        for (TokenRangeReplicasResponse.ReplicaInfo r: writeReplicas)
+        {
+            Range<BigInteger> range = Range.openClosed(BigInteger.valueOf(Long.parseLong(r.start())),
+                                                       BigInteger.valueOf(Long.parseLong(r.end())));
+            assertThat(expectedRangeMapping).containsKey("datacenter1");
+            assertThat(expectedRangeMapping.get("datacenter1")).containsKey(range);
+            // Replicaset for the same range match expected
+            assertThat(r.replicasByDatacenter().get("datacenter1"))
+            .containsExactlyInAnyOrderElementsOf(expectedRangeMapping.get("datacenter1").get(range));
+
+            if (annotation.numDcs() > 1)
+            {
+                assertThat(expectedRangeMapping).containsKey("datacenter2");
+                assertThat(expectedRangeMapping.get("datacenter2")).containsKey(range);
+                assertThat(r.replicasByDatacenter().get("datacenter2"))
+                .containsExactlyInAnyOrderElementsOf(expectedRangeMapping.get("datacenter2").get(range));
+            }
+        }
+    }
+
+    void retrieveMappingWithKeyspace(VertxTestContext context, String keyspace,
+                                     Handler<HttpResponse<Buffer>> verifier) throws Exception
+    {
+        String testRoute = "/api/v1/keyspaces/" + keyspace + "/token-range-replicas";
+        testWithClient(context, client -> {
+            client.get(server.actualPort(), "127.0.0.1", testRoute)
+                  .send(context.succeeding(verifier));
+        });
+    }
+
+    void assertMappingResponseOK(TokenRangeReplicasResponse mappingResponse)
+    {
+        assertMappingResponseOK(mappingResponse, 1, Collections.singleton("datacenter1"));
+    }
+
+    void assertMappingResponseOK(TokenRangeReplicasResponse mappingResponse,
+                                 int replicationFactor,
+                                 Set<String> dcReplication)
+    {
+        assertThat(mappingResponse).isNotNull();
+        assertThat(mappingResponse.readReplicas()).isNotNull();
+        assertThat(mappingResponse.writeReplicas()).isNotNull();
+        TokenRangeReplicasResponse.ReplicaInfo readReplica = mappingResponse.readReplicas().get(0);
+        assertThat(readReplica.replicasByDatacenter()).isNotNull().hasSize(dcReplication.size());
+        TokenRangeReplicasResponse.ReplicaInfo writeReplica = mappingResponse.writeReplicas().get(0);
+        assertThat(writeReplica.replicasByDatacenter()).isNotNull().hasSize(dcReplication.size());
+
+        for (String dcName : dcReplication)
+        {
+            assertThat(readReplica.replicasByDatacenter().keySet()).isNotEmpty().contains(dcName);
+            assertThat(readReplica.replicasByDatacenter().get(dcName)).isNotNull().hasSize(replicationFactor);
+
+            assertThat(writeReplica.replicasByDatacenter().keySet()).isNotEmpty().contains(dcName);
+            assertThat(writeReplica.replicasByDatacenter().get(dcName)).isNotNull();
+            assertThat(writeReplica.replicasByDatacenter().get(dcName).size())
+            .isGreaterThanOrEqualTo(replicationFactor);
+        }
+    }
+
+    /**
+     * ByteBuddy helper for a single joining node
+     */
+    @Shared
+    public static class BBHelperSingleJoiningNode
+    {
+        public static final CountDownLatch TRANSIENT_STATE_START = new CountDownLatch(1);
+        public static final CountDownLatch TRANSIENT_STATE_END = new CountDownLatch(1);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 3 node cluster with 1 joining node
+            // We intercept the bootstrap of the leaving node (4) to validate token ranges
+            if (nodeNumber == 6)
+            {
+                TypePool typePool = TypePool.Default.of(cl);
+                TypeDescription description = typePool.describe("org.apache.cassandra.service.StorageService")
+                                                      .resolve();
+                new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
+                               .method(named("bootstrap").and(takesArguments(2)))
+                               .intercept(MethodDelegation.to(BBHelperSingleJoiningNode.class))
+                               // Defer class loading until all dependencies are loaded
+                               .make(TypeResolutionStrategy.Lazy.INSTANCE, typePool)
+                               .load(cl, ClassLoadingStrategy.Default.INJECTION);
+            }
+        }
+
+        public static boolean bootstrap(Collection<?> tokens,
+                                        long bootstrapTimeoutMillis,
+                                        @SuperCall Callable<Boolean> orig) throws Exception
+        {
+            boolean result = orig.call();
+            // trigger bootstrap start and wait until bootstrap is ready from test
+            TRANSIENT_STATE_START.countDown();
+            Uninterruptibles.awaitUninterruptibly(TRANSIENT_STATE_END);
+            return result;
+        }
+    }
+
+    /**
+     * ByteBuddy helper for multiple joining nodes
+     */
+    @Shared
+    public static class BBHelperMultipleJoiningNodes
+    {
+        public static final CountDownLatch TRANSIENT_STATE_START = new CountDownLatch(2);
+        public static final CountDownLatch TRANSIENT_STATE_END = new CountDownLatch(2);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 3 node cluster with a 2 joining nodes
+            // We intercept the joining of nodes (4, 5) to validate token ranges
+            if (nodeNumber > 3)
+            {
+                TypePool typePool = TypePool.Default.of(cl);
+                TypeDescription description = typePool.describe("org.apache.cassandra.service.StorageService")
+                                                      .resolve();
+                new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
+                               .method(named("bootstrap").and(takesArguments(2)))
+                               .intercept(MethodDelegation.to(BBHelperMultipleJoiningNodes.class))
+                               // Defer class loading until all dependencies are loaded
+                               .make(TypeResolutionStrategy.Lazy.INSTANCE, typePool)
+                               .load(cl, ClassLoadingStrategy.Default.INJECTION);
+            }
+        }
+
+        public static boolean bootstrap(Collection<?> tokens,
+                                        long bootstrapTimeoutMillis,
+                                        @SuperCall Callable<Boolean> orig) throws Exception
+        {
+            boolean result = orig.call();
+            // trigger bootstrap start and wait until bootstrap is ready from test
+            TRANSIENT_STATE_START.countDown();
+            Uninterruptibles.awaitUninterruptibly(TRANSIENT_STATE_END);
+            return result;
+        }
+    }
+
+    /**
+     * ByteBuddy helper for doubling cluster size
+     */
+    @Shared
+    public static class BBHelperDoubleClusterSize
+    {
+        public static final CountDownLatch TRANSIENT_STATE_START = new CountDownLatch(5);
+        public static final CountDownLatch TRANSIENT_STATE_END = new CountDownLatch(5);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 5 node cluster doubling in size
+            // We intercept the bootstrap of the new nodes (6-10) to validate token ranges
+            if (nodeNumber > 5)
+            {
+                TypePool typePool = TypePool.Default.of(cl);
+                TypeDescription description = typePool.describe("org.apache.cassandra.service.StorageService")
+                                                      .resolve();
+                new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
+                               .method(named("bootstrap").and(takesArguments(2)))
+                               .intercept(MethodDelegation.to(BBHelperDoubleClusterSize.class))
+                               // Defer class loading until all dependencies are loaded
+                               .make(TypeResolutionStrategy.Lazy.INSTANCE, typePool)
+                               .load(cl, ClassLoadingStrategy.Default.INJECTION);
+            }
+        }
+
+        public static boolean bootstrap(Collection<?> tokens,
+                                        long bootstrapTimeoutMillis,
+                                        @SuperCall Callable<Boolean> orig) throws Exception
+        {
+            boolean result = orig.call();
+            // trigger bootstrap start and wait until bootstrap is ready from test
+            TRANSIENT_STATE_START.countDown();
+            Uninterruptibles.awaitUninterruptibly(TRANSIENT_STATE_END);
+            return result;
+        }
+    }
+
+    /**
+     * ByteBuddy Helper for a single leaving node
+     */
+    @Shared
+    public static class BBHelperSingleLeavingNode
+    {
+        public static final CountDownLatch TRANSIENT_STATE_START = new CountDownLatch(1);
+        public static final CountDownLatch TRANSIENT_STATE_END = new CountDownLatch(1);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 5 node cluster with 1 leaving node
+            // We intercept the shutdown of the leaving node (5) to validate token ranges
+            if (nodeNumber == 5)
+            {
+                TypePool typePool = TypePool.Default.of(cl);
+                TypeDescription description = typePool.describe("org.apache.cassandra.service.StorageService")
+                                                      .resolve();
+                new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
+                               .method(named("unbootstrap"))
+                               .intercept(MethodDelegation.to(BBHelperSingleLeavingNode.class))
+                               // Defer class loading until all dependencies are loaded
+                               .make(TypeResolutionStrategy.Lazy.INSTANCE, typePool)
+                               .load(cl, ClassLoadingStrategy.Default.INJECTION);
+            }
+        }
+
+        @SuppressWarnings("unused")
+        public static void unbootstrap(@SuperCall Callable<?> orig) throws Exception
+        {
+            TRANSIENT_STATE_START.countDown();
+            Uninterruptibles.awaitUninterruptibly(TRANSIENT_STATE_END);
+            orig.call();
+        }
+    }
+
+    /**
+     * ByteBuddy helper for multiple leaving nodes
+     */
+    @Shared
+    public static class BBHelperMultipleLeavingNodes

Review Comment:
   Each BBHelper class is used in a single test class, and their implementations (install) highly depends on the test class, e.g. nodeNumber assumption. Therefore, it makes it hard to share the implementation with other test classes. How about move it to the test class where it is actually used? 



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/Partitioner.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+
+/**
+ * Represents types of Partitioners supported and the corresponding starting token values
+ */
+public enum Partitioner
+{
+    Murmur3(BigInteger.valueOf(Long.MIN_VALUE), BigInteger.valueOf(Long.MAX_VALUE)),
+    Random(BigInteger.ONE.negate(), BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE));

Review Comment:
   Second to this. The value range of random partition token should be the same as in Cassandra.
   
   Besides that, can you add a brief description to clarify the range's ends. The minimum token is not inclusive, and the maximum token is. In math notion, it is `(MIN, MAX]`



##########
src/test/resources/logback-test.xml:
##########
@@ -20,7 +20,7 @@
 
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
     <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
-      <level>DEBUG</level>
+      <level>INFO</level>

Review Comment:
   I think `DEBUG` is more suitable for tests. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1310534787


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     naturalReplicaMappings);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        LOGGER.debug("Pending token range mappings for keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     pendingRangeMappings);
+        List<TokenRangeReplicas> naturalTokenRangeReplicas = transformRangeMappings(naturalReplicaMappings,
+                                                                                    partitioner);
+        List<TokenRangeReplicas> pendingTokenRangeReplicas = transformRangeMappings(pendingRangeMappings,
+                                                                                    partitioner);
+
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> allTokenRangeReplicas = new ArrayList<>(naturalTokenRangeReplicas);
+        allTokenRangeReplicas.addAll(pendingTokenRangeReplicas);
+
+        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(allTokenRangeReplicas);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<ReplicaInfo> writeReplicas = writeReplicasFromPendingRanges(allTokenRangeReplicas, hostToDatacenter);
+
+        List<ReplicaInfo> readReplicas = readReplicasFromReplicaMapping(naturalTokenRangeReplicas, hostToDatacenter);
+        Map<String, String> replicaToStateMap = replicaToStateMap(allTokenRangeReplicas, storage);
+
+        return new TokenRangeReplicasResponse(replicaToStateMap,
+                                              writeReplicas,
+                                              readReplicas);
+    }
+
+    private Map<String, String> replicaToStateMap(List<TokenRangeReplicas> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    protected EndpointSnitchJmxOperations initializeEndpointProxy()
+    {
+        return jmxClient.proxy(EndpointSnitchJmxOperations.class, ENDPOINT_SNITCH_INFO_OBJ_NAME);
+    }
+
+    protected StorageJmxOperations initializeStorageOps()
+    {
+        return jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+    }
+
+
+    protected String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<ReplicaInfo> writeReplicasFromPendingRanges(List<TokenRangeReplicas> tokenRangeReplicaSet,
+                                                             Map<String, String> hostToDatacenter)
+    {
+//        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(tokenRangeReplicaSet);
+        // Candidate write-replica mappings are normalized by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(tokenRangeReplicaSet).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new ReplicaInfo(range.start().toString(),
+                                                            range.end().toString(),
+                                                            replicasByDc);
+                                 })
+                                 .collect(toList());
+    }
+
+    private List<TokenRangeReplicas> transformRangeMappings(Map<List<String>, List<String>> replicaMappings,
+                                                            Partitioner partitioner)
+    {
+        return replicaMappings.entrySet()
+                              .stream()
+                              .map(entry -> generateTokenRangeReplicas(new BigInteger(entry.getKey().get(0)),
+                                                                       new BigInteger(entry.getKey().get(1)),
+                                                                       partitioner,
+                                                                       new HashSet<>(entry.getValue())))
+                              .flatMap(Collection::stream)
+                              .collect(toList());
+    }
+
+
+    private List<ReplicaInfo> readReplicasFromReplicaMapping(List<TokenRangeReplicas> naturalTokenRangeReplicas,
+                                                             Map<String, String> hostToDatacenter)
+    {
+        Map<String, String> hostToDatacenter2 = buildHostToDatacenterMapping(naturalTokenRangeReplicas);
+        return naturalTokenRangeReplicas.stream()
+                                        .sorted()
+                                        .map(rep -> {
+                                            Map<String, List<String>> replicasByDc
+                                            = replicasByDataCenter(hostToDatacenter2, rep.replicaSet());
+
+                                            return new ReplicaInfo(rep.start().toString(),
+                                                                   rep.end().toString(),
+                                                                   replicasByDc);
+                                        })
+                                        .collect(toList());
+    }
+
+    private Map<String, String> buildHostToDatacenterMapping(List<TokenRangeReplicas> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = initializeEndpointProxy();
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get,
+                                                               Collectors.filtering(replicas::contains, toList())));

Review Comment:
   Addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307963708


##########
src/test/integration/org/apache/cassandra/sidecar/routes/MultiDcTokenSupplier.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.distributed.api.TokenSupplier;
+
+/**
+ * Static factory holder that provides a multi-DC token supplier
+ */
+public class MultiDcTokenSupplier
+{
+
+    static TokenSupplier evenlyDistributedTokens(int numNodes, int numDcs, int numTokens)
+    {
+        long totalTokens = (long) numNodes * numDcs * numTokens;
+        BigInteger increment = BigInteger.valueOf((Long.MAX_VALUE / totalTokens) * 4);

Review Comment:
   This was hard-coded to account for the no. DCs. Will update to be as folllows. `2` is as-is from the existing `TokenSupplier` (to account for the token-space being from Long.MIN - Long.MAX).
   
   `(Long.MAX_VALUE / totalTokens)* 2 * numDCs`
   
   We broaden the increment space by numDcs as the tokens assigned to neighboring nodes across different DCs are consecutive (i.e. delta 1).
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1309239449


##########
src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java:
##########
@@ -148,34 +149,21 @@ private BeforeEachCallback beforeEach()
                 int nodesPerDc = annotation.nodesPerDc();
                 int dcCount = annotation.numDcs();
                 int newNodesPerDc = annotation.newNodesPerDc(); // if the test wants to add more nodes later
+                Preconditions.checkArgument(newNodesPerDc >= 0,
+                                            "newNodesPerDc cannot be a negative number");
+                int originalNodeCount = nodesPerDc * dcCount;
                 int finalNodeCount = dcCount * (nodesPerDc + newNodesPerDc);
                 Versions.Version requestedVersion = versions.getLatest(new Semver(version.version(),
                                                                                   Semver.SemverType.LOOSE));
                 SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(version.version());
 
                 UpgradeableCluster.Builder clusterBuilder =
-                    UpgradeableCluster.build(nodesPerDc * dcCount)
-                                      .withVersion(requestedVersion)
-                                      .withDCs(dcCount)
-                                      .withDataDirCount(annotation.numDataDirsPerInstance())
-                                      .withConfig(config -> {
-                                      if (annotation.nativeTransport())
-                                      {
-                                          config.with(Feature.NATIVE_PROTOCOL);
-                                      }
-                                      if (annotation.jmx())
-                                      {
-                                          config.with(Feature.JMX);
-                                      }
-                                      if (annotation.gossip())
-                                      {
-                                          config.with(Feature.GOSSIP);
-                                      }
-                                      if (annotation.network())
-                                      {
-                                          config.with(Feature.NETWORK);
-                                      }
-                                  });
+                UpgradeableCluster.build(originalNodeCount)
+//                                  .withDynamicPortAllocation(true) // to allow parallel test runs

Review Comment:
   Thanks. will update and enable parallel runs



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313509401


##########
client/src/testFixtures/java/org/apache/cassandra/sidecar/client/SidecarClientTest.java:
##########
@@ -255,6 +256,31 @@ public void testTimeSkewFromReplicaSet() throws Exception
         validateResponseServed(ApiEndpointsV1.TIME_SKEW_ROUTE);
     }
 
+    @Test
+    public void testTokenRangeReplicasFromReplicaSet() throws Exception
+    {
+        String keyspace = "test";
+        String tokenRangeReplicasAsString = "{\"replicaState\":{" +
+                                            "\"127.0.0.1:7000\":\"NORMAL\"}," +
+                                            "\"writeReplicas\":[{\"start\":\"-9223372036854775808\"," +
+                                            "\"end\":\"9223372036854775807\",\"replicasByDatacenter\":" +
+                                            "{\"datacenter1\":[\"127.0.0.1:7000\"]}}],\"readReplicas\":" +
+                                            "[{\"start\":\"-9223372036854775808\",\"end\":\"9223372036854775807\"," +
+                                            "\"replicasByDatacenter\":{\"datacenter1\":[\"127.0.0.1:7000\"]}}]}";
+        MockResponse response = new MockResponse().setResponseCode(OK.code()).setBody(tokenRangeReplicasAsString);
+        enqueue(response);
+
+        TokenRangeReplicasResponse result = client.tokenRangeReplicas(instances.subList(1, 2), keyspace)
+                                                  .get(30, TimeUnit.SECONDS);
+        assertThat(result).isNotNull();
+        assertThat(result.writeReplicas()).hasSize(1);

Review Comment:
   Addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307699965


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token

Review Comment:
   Makes sense. Seems like we're no longer using this method with the new token-range intersection resolution algorithm. Will remove.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1306067825


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)

Review Comment:
   Same (make things protected) goes for any members that need to be accessible by the derived class (or provide a protected getter if you feel that's necessary, but I think it's mostly the `private final JmxClient jmxClient`, which would be OK to be protected).



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)

Review Comment:
   This method is misnamed - it doesn't group anything, it just enriches the host with a datacenter - maybe just `buildHostToDatacenterMapping`?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";
+        private final Set<String> joiningNodes;
+        private final GossipInfoResponse gossipInfo;
+
+        StateWithReplacement(List<String> joiningNodes, List<String> leavingNodes, List<String> movingNodes,
+                             GossipInfoResponse gossipInfo)
+        {
+            super(joiningNodes, leavingNodes, movingNodes);
+            this.joiningNodes = new HashSet<>(joiningNodes);
+            this.gossipInfo = gossipInfo;
+        }
+
+        /**
+         * This method returns state of a node and accounts for a new 'Replacing' state if the node is
+         * replacing a dead node. For returning this state, the method checks status of the node in gossip
+         * information.
+         *
+         * @param endpoint node information represented usually in form of 'ip:port'
+         * @return Node status
+         */
+        @Override
+        String of(String endpoint)
+        {
+            if (joiningNodes.contains(endpoint))
+            {
+                GossipInfoResponse.GossipInfo gossipInfoEntry = gossipInfo.get(endpoint);
+
+                if (gossipInfoEntry != null)
+                {
+                    LOGGER.info("Found gossipInfoEntry={}", gossipInfoEntry);

Review Comment:
   NIT: Debug?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());

Review Comment:
   The indentation/formatting here is horrible to follow - I realize this is the default the project uses, but is there a way we can improve this, as it's really quite difficult to understand what's at what level of the concat/map/etc. here...
   Alternately, break this down into smaller chunks and assign each to an intermediate variable with a good name.



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);

Review Comment:
   General comment about this line of code + most of the rest of this class - It's _really hard_ to figure out what's what in this class with everything being maps of native types - I realize that we can't change the return types of the JMX endpoints, but can we please try to transform things as soon as possible after we get them to something more meaningful types (Datacenter, Range, ReplicaInfo, Token, some of which I realize we don't have yet)? It would make all of this logic significantly easier to understand, and prevent bugs in the future.



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token

Review Comment:
   This assumption should be checked with a precondition, or we should write the method to not care about if `this` comes before `other`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308013485


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";
+        private final Set<String> joiningNodes;

Review Comment:
   Agreed. Will update.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok closed pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok closed pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…
URL: https://github.com/apache/cassandra-sidecar/pull/58


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313667589


##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());

Review Comment:
   yep, redundant and not required. will remove



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307678918


##########
build.gradle:
##########
@@ -326,6 +326,8 @@ tasks.register("integrationTest", Test) {
     useJUnitPlatform() {
         includeTags "integrationTest"
     }
+// Uncomment below to run unit tests in parallel
+//    maxParallelForks = Runtime.runtime.availableProcessors() * 2

Review Comment:
   It requires a Cassandra fix (https://issues.apache.org/jira/browse/CASSANDRA-18722) that isn't yet committed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308014067


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)

Review Comment:
   Yep, not needed. Will switch to package-private for tests



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308152584


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     naturalReplicaMappings);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        LOGGER.debug("Pending token range mappings for keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     pendingRangeMappings);
+        List<TokenRangeReplicas> naturalTokenRangeReplicas = transformRangeMappings(naturalReplicaMappings,
+                                                                                    partitioner);
+        List<TokenRangeReplicas> pendingTokenRangeReplicas = transformRangeMappings(pendingRangeMappings,
+                                                                                    partitioner);
+
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> allTokenRangeReplicas = new ArrayList<>(naturalTokenRangeReplicas);
+        allTokenRangeReplicas.addAll(pendingTokenRangeReplicas);
+
+        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(allTokenRangeReplicas);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<ReplicaInfo> writeReplicas = writeReplicasFromPendingRanges(allTokenRangeReplicas, hostToDatacenter);
+
+        List<ReplicaInfo> readReplicas = readReplicasFromReplicaMapping(naturalTokenRangeReplicas, hostToDatacenter);
+        Map<String, String> replicaToStateMap = replicaToStateMap(allTokenRangeReplicas, storage);
+
+        return new TokenRangeReplicasResponse(replicaToStateMap,
+                                              writeReplicas,
+                                              readReplicas);
+    }
+
+    private Map<String, String> replicaToStateMap(List<TokenRangeReplicas> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    protected EndpointSnitchJmxOperations initializeEndpointProxy()
+    {
+        return jmxClient.proxy(EndpointSnitchJmxOperations.class, ENDPOINT_SNITCH_INFO_OBJ_NAME);
+    }
+
+    protected StorageJmxOperations initializeStorageOps()
+    {
+        return jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+    }
+
+
+    protected String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<ReplicaInfo> writeReplicasFromPendingRanges(List<TokenRangeReplicas> tokenRangeReplicaSet,
+                                                             Map<String, String> hostToDatacenter)
+    {
+//        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(tokenRangeReplicaSet);
+        // Candidate write-replica mappings are normalized by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(tokenRangeReplicaSet).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new ReplicaInfo(range.start().toString(),
+                                                            range.end().toString(),
+                                                            replicasByDc);
+                                 })
+                                 .collect(toList());
+    }
+
+    private List<TokenRangeReplicas> transformRangeMappings(Map<List<String>, List<String>> replicaMappings,
+                                                            Partitioner partitioner)
+    {
+        return replicaMappings.entrySet()
+                              .stream()
+                              .map(entry -> generateTokenRangeReplicas(new BigInteger(entry.getKey().get(0)),
+                                                                       new BigInteger(entry.getKey().get(1)),
+                                                                       partitioner,
+                                                                       new HashSet<>(entry.getValue())))
+                              .flatMap(Collection::stream)
+                              .collect(toList());
+    }
+
+
+    private List<ReplicaInfo> readReplicasFromReplicaMapping(List<TokenRangeReplicas> naturalTokenRangeReplicas,
+                                                             Map<String, String> hostToDatacenter)

Review Comment:
   unused variable. 



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     naturalReplicaMappings);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        LOGGER.debug("Pending token range mappings for keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     pendingRangeMappings);
+        List<TokenRangeReplicas> naturalTokenRangeReplicas = transformRangeMappings(naturalReplicaMappings,
+                                                                                    partitioner);
+        List<TokenRangeReplicas> pendingTokenRangeReplicas = transformRangeMappings(pendingRangeMappings,
+                                                                                    partitioner);
+
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> allTokenRangeReplicas = new ArrayList<>(naturalTokenRangeReplicas);
+        allTokenRangeReplicas.addAll(pendingTokenRangeReplicas);
+
+        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(allTokenRangeReplicas);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<ReplicaInfo> writeReplicas = writeReplicasFromPendingRanges(allTokenRangeReplicas, hostToDatacenter);
+
+        List<ReplicaInfo> readReplicas = readReplicasFromReplicaMapping(naturalTokenRangeReplicas, hostToDatacenter);
+        Map<String, String> replicaToStateMap = replicaToStateMap(allTokenRangeReplicas, storage);
+
+        return new TokenRangeReplicasResponse(replicaToStateMap,
+                                              writeReplicas,
+                                              readReplicas);
+    }
+
+    private Map<String, String> replicaToStateMap(List<TokenRangeReplicas> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    protected EndpointSnitchJmxOperations initializeEndpointProxy()
+    {
+        return jmxClient.proxy(EndpointSnitchJmxOperations.class, ENDPOINT_SNITCH_INFO_OBJ_NAME);
+    }
+
+    protected StorageJmxOperations initializeStorageOps()
+    {
+        return jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+    }
+
+
+    protected String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<ReplicaInfo> writeReplicasFromPendingRanges(List<TokenRangeReplicas> tokenRangeReplicaSet,
+                                                             Map<String, String> hostToDatacenter)
+    {
+//        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(tokenRangeReplicaSet);

Review Comment:
   rm this?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     naturalReplicaMappings);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        LOGGER.debug("Pending token range mappings for keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     pendingRangeMappings);
+        List<TokenRangeReplicas> naturalTokenRangeReplicas = transformRangeMappings(naturalReplicaMappings,
+                                                                                    partitioner);
+        List<TokenRangeReplicas> pendingTokenRangeReplicas = transformRangeMappings(pendingRangeMappings,
+                                                                                    partitioner);
+
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> allTokenRangeReplicas = new ArrayList<>(naturalTokenRangeReplicas);
+        allTokenRangeReplicas.addAll(pendingTokenRangeReplicas);
+
+        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(allTokenRangeReplicas);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<ReplicaInfo> writeReplicas = writeReplicasFromPendingRanges(allTokenRangeReplicas, hostToDatacenter);
+
+        List<ReplicaInfo> readReplicas = readReplicasFromReplicaMapping(naturalTokenRangeReplicas, hostToDatacenter);
+        Map<String, String> replicaToStateMap = replicaToStateMap(allTokenRangeReplicas, storage);
+
+        return new TokenRangeReplicasResponse(replicaToStateMap,
+                                              writeReplicas,
+                                              readReplicas);
+    }
+
+    private Map<String, String> replicaToStateMap(List<TokenRangeReplicas> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    protected EndpointSnitchJmxOperations initializeEndpointProxy()
+    {
+        return jmxClient.proxy(EndpointSnitchJmxOperations.class, ENDPOINT_SNITCH_INFO_OBJ_NAME);
+    }
+
+    protected StorageJmxOperations initializeStorageOps()
+    {
+        return jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+    }
+
+
+    protected String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<ReplicaInfo> writeReplicasFromPendingRanges(List<TokenRangeReplicas> tokenRangeReplicaSet,
+                                                             Map<String, String> hostToDatacenter)
+    {
+//        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(tokenRangeReplicaSet);
+        // Candidate write-replica mappings are normalized by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(tokenRangeReplicaSet).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new ReplicaInfo(range.start().toString(),
+                                                            range.end().toString(),
+                                                            replicasByDc);
+                                 })
+                                 .collect(toList());
+    }
+
+    private List<TokenRangeReplicas> transformRangeMappings(Map<List<String>, List<String>> replicaMappings,
+                                                            Partitioner partitioner)
+    {
+        return replicaMappings.entrySet()
+                              .stream()
+                              .map(entry -> generateTokenRangeReplicas(new BigInteger(entry.getKey().get(0)),
+                                                                       new BigInteger(entry.getKey().get(1)),
+                                                                       partitioner,
+                                                                       new HashSet<>(entry.getValue())))
+                              .flatMap(Collection::stream)
+                              .collect(toList());
+    }
+
+
+    private List<ReplicaInfo> readReplicasFromReplicaMapping(List<TokenRangeReplicas> naturalTokenRangeReplicas,
+                                                             Map<String, String> hostToDatacenter)
+    {
+        Map<String, String> hostToDatacenter2 = buildHostToDatacenterMapping(naturalTokenRangeReplicas);
+        return naturalTokenRangeReplicas.stream()
+                                        .sorted()
+                                        .map(rep -> {
+                                            Map<String, List<String>> replicasByDc
+                                            = replicasByDataCenter(hostToDatacenter2, rep.replicaSet());
+
+                                            return new ReplicaInfo(rep.start().toString(),
+                                                                   rep.end().toString(),
+                                                                   replicasByDc);
+                                        })
+                                        .collect(toList());
+    }
+
+    private Map<String, String> buildHostToDatacenterMapping(List<TokenRangeReplicas> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = initializeEndpointProxy();
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get,
+                                                               Collectors.filtering(replicas::contains, toList())));

Review Comment:
   The `adapter` subproject declare source compatibility of 1.8, but `Collectors.filtering` is available after java 9.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307958712


##########
common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java:
##########
@@ -259,12 +259,16 @@ private Map<String, Object> buildJmxEnv()
     }
 
     @Override
-    public synchronized void close() throws IOException
+    public void close() throws IOException
     {
-        JMXConnector connector = jmxConnector;
-        if (connector != null)
+        JMXConnector connector;
+        synchronized (this)
         {
+            connector = jmxConnector;
             jmxConnector = null;

Review Comment:
   Good catch. This would need to be reset to false to prevent false-positives from `checkConnection`. Will add.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1310545898


##########
src/test/resources/logback-test.xml:
##########
@@ -20,7 +20,7 @@
 
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
     <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
-      <level>INFO</level>
+      <level>DEBUG</level>

Review Comment:
   Typically `Debug` is used when running test. What are the other scenarios of using debug? It is (almost) never considered in production. So.. if we do not really enable debug level, what is the point of having those debug message in the code :D
   I think the way too many log messages is a different issue. Some useless chatty logger can be disabled or set to INFO. 
   We need to put more thoughts. For now, I am fine with "INFO" if Francisco feels strong about it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313357874


##########
README.md:
##########
@@ -49,6 +49,19 @@ The build script supports two parameters:
 Remove any versions you may not want to test with. We recommend at least the latest (released) 4.X series and `trunk`.
 See Testing for more details on how to choose which Cassandra versions to use while testing.
 
+For multi-node in-jvm dtests, network aliases will need to be setup for each Cassandra node. The tests assume each node's 
+ip address is 127.0.0.x, where x is the node id. 
+
+For example if you populated your cluster with 3 nodes, create interfaces for 127.0.0.2 and 127.0.0.3 (the first node of course uses 127.0.0.1).
+
+### macOS network aliases
+To get up and running, create a temporary alias for every node except the first:
+
+```
+sudo ifconfig lo0 alias 127.0.0.2

Review Comment:
   Note: There are tests in the multi-dc scenarios that go up to .12, so please increase the end of the range to at least 12 (maybe 20?).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312416071


##########
client/src/testFixtures/java/org/apache/cassandra/sidecar/client/SidecarClientTest.java:
##########
@@ -255,6 +256,31 @@ public void testTimeSkewFromReplicaSet() throws Exception
         validateResponseServed(ApiEndpointsV1.TIME_SKEW_ROUTE);
     }
 
+    @Test
+    public void testTokenRangeReplicasFromReplicaSet() throws Exception
+    {
+        String keyspace = "test";
+        String tokenRangeReplicasAsString = "{\"replicaState\":{" +
+                                            "\"127.0.0.1:7000\":\"NORMAL\"}," +
+                                            "\"writeReplicas\":[{\"start\":\"-9223372036854775808\"," +
+                                            "\"end\":\"9223372036854775807\",\"replicasByDatacenter\":" +
+                                            "{\"datacenter1\":[\"127.0.0.1:7000\"]}}],\"readReplicas\":" +
+                                            "[{\"start\":\"-9223372036854775808\",\"end\":\"9223372036854775807\"," +
+                                            "\"replicasByDatacenter\":{\"datacenter1\":[\"127.0.0.1:7000\"]}}]}";
+        MockResponse response = new MockResponse().setResponseCode(OK.code()).setBody(tokenRangeReplicasAsString);
+        enqueue(response);
+
+        TokenRangeReplicasResponse result = client.tokenRangeReplicas(instances.subList(1, 2), keyspace)
+                                                  .get(30, TimeUnit.SECONDS);
+        assertThat(result).isNotNull();
+        assertThat(result.writeReplicas()).hasSize(1);

Review Comment:
   As we've discussed in other places - can we validate that the data the mock server responded with was actually what we got? It's good to make sure there aren't any JSON serialization issues by actually validating everything here rather than just the shape of the response.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307685706


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);

Review Comment:
   As you have called out, the native types are the return types of the JMX endpoints. I am transforming this into a holder `TokenRangeReplicas` further in the flow. Will look into moving it ahead.



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)

Review Comment:
   Will make the change similar to how RingHandler does this for consistency



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#issuecomment-1719997735

   Changes were merged into trunk. Closing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312439641


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token
+     *
+     * @param other the next range in the range list to compare
+     * @return true if "this" range is larger than the other
+     */
+    protected boolean isLarger(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return this.end.subtract(this.start).compareTo(other.end.subtract(other.start)) > 0;
+    }
+
+    /**
+     * Determines intersection if the next range starts before the current range ends. This method assumes that
+     * the provided ranges are sorted and unwrapped.
+     * When the current range goes all the way to the end, we determine intersection if the next range starts
+     * after the current since all subsequent ranges have to be subsets.
+     *
+     * @param other the range we are currently processing to check if "this" intersects it
+     * @return true if "this" range intersects the other
+     */
+    protected boolean intersects(TokenRangeReplicas other)
+    {
+        if (this.compareTo(other) > 0)
+            throw new IllegalStateException(
+            String.format("Token ranges - (this:%s other:%s) are not ordered", this, other));
+
+        return this.end.compareTo(other.start) > 0 && this.start.compareTo(other.end) < 0; // Start exclusive (DONE)
+    }

Review Comment:
   Apart from here there are other operations (eg. `processIntersectingRanges`) that rely on the ranges being processed to be sorted and breaks if the prerequisite is not met. Although that should "never" happen since we are explicitly sorting and unwrapping upfront. 
   
   Like you mentioned earlier, I'm leaning towards performing the assertions on the assumptions up-front, if for any reason the ranges are not what we expect them to be.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1309084094


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/Partitioner.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+
+/**
+ * Represents types of Partitioners supported and the corresponding starting token values
+ */
+public enum Partitioner
+{
+    Murmur3(BigInteger.valueOf(Long.MIN_VALUE), BigInteger.valueOf(Long.MAX_VALUE)),
+    Random(BigInteger.ONE.negate(), BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE));

Review Comment:
   Updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1309092430


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.adapters.base;
+
+/**
+ * Holder class for Node related
+ */
+public class NodeInfo

Review Comment:
   I just don't see the benefit, I'm not against or for it, I'm neutral. I would not use an enum if we are just providing constants. This is what Cassandra does for ring: https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/tools/nodetool/Ring.java#L172



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1310535140


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     naturalReplicaMappings);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        LOGGER.debug("Pending token range mappings for keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     pendingRangeMappings);
+        List<TokenRangeReplicas> naturalTokenRangeReplicas = transformRangeMappings(naturalReplicaMappings,
+                                                                                    partitioner);
+        List<TokenRangeReplicas> pendingTokenRangeReplicas = transformRangeMappings(pendingRangeMappings,
+                                                                                    partitioner);
+
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> allTokenRangeReplicas = new ArrayList<>(naturalTokenRangeReplicas);
+        allTokenRangeReplicas.addAll(pendingTokenRangeReplicas);
+
+        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(allTokenRangeReplicas);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<ReplicaInfo> writeReplicas = writeReplicasFromPendingRanges(allTokenRangeReplicas, hostToDatacenter);
+
+        List<ReplicaInfo> readReplicas = readReplicasFromReplicaMapping(naturalTokenRangeReplicas, hostToDatacenter);
+        Map<String, String> replicaToStateMap = replicaToStateMap(allTokenRangeReplicas, storage);
+
+        return new TokenRangeReplicasResponse(replicaToStateMap,
+                                              writeReplicas,
+                                              readReplicas);
+    }
+
+    private Map<String, String> replicaToStateMap(List<TokenRangeReplicas> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    protected EndpointSnitchJmxOperations initializeEndpointProxy()
+    {
+        return jmxClient.proxy(EndpointSnitchJmxOperations.class, ENDPOINT_SNITCH_INFO_OBJ_NAME);
+    }
+
+    protected StorageJmxOperations initializeStorageOps()
+    {
+        return jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+    }
+
+
+    protected String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<ReplicaInfo> writeReplicasFromPendingRanges(List<TokenRangeReplicas> tokenRangeReplicaSet,
+                                                             Map<String, String> hostToDatacenter)
+    {
+//        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(tokenRangeReplicaSet);

Review Comment:
   Addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1300512576


##########
src/test/integration/org/apache/cassandra/sidecar/routes/TokenRangeIntegrationMovingTest.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.ClusterUtils;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Node movement scenarios integration tests for token range replica mapping endpoint with cassandra container.
+ */
+@ExtendWith(VertxExtension.class)
+public class TokenRangeIntegrationMovingTest extends BaseTokenRangeIntegrationTest
+{
+    private static final int MOVING_NODE_IDX = 5;
+
+    @CassandraIntegrationTest(nodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithKeyspaceMovingNode(VertxTestContext context,
+                                               ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+
+        UpgradeableCluster cluster =
+        cassandraTestContext.configureAndStartCluster(builder ->
+                                                      builder.withInstanceInitializer(BBHelperMovingNode::install));
+        runMovingTestScenario(context,
+                              BBHelperMovingNode.TRANSIENT_STATE_START,
+                              BBHelperMovingNode.TRANSIENT_STATE_END,
+                              cluster);
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 5, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWhileMovingNodeMultiDC(VertxTestContext context,
+                                               ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier = MultiDcTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                      annotation.numDcs(),
+                                                                                      1);
+        UpgradeableCluster cluster =
+        cassandraTestContext.configureAndStartCluster(builder -> {
+                                                          builder.withInstanceInitializer(BBHelperMovingNode::install);
+                                                          builder.withTokenSupplier(mdcTokenSupplier);
+                                                      }
+        );
+
+        runMovingTestScenario(context,
+                              BBHelperMovingNode.TRANSIENT_STATE_START,
+                              BBHelperMovingNode.TRANSIENT_STATE_END,
+                              cluster);
+    }
+
+    // TODO: Multiple replica-safe node movements in same DC, different DCs

Review Comment:
   Will remove and add more coverage for multiple movements



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307958167


##########
common/build.gradle:
##########
@@ -41,6 +41,8 @@ repositories {
 
 test {
     useJUnitPlatform()
+// Uncomment below to run unit tests in parallel
+//    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1

Review Comment:
   This was introduced by @frankgh in a prior commit to my branch, so I will let him respond. Leaving it out of this commit for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308792310


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.adapters.base;
+
+/**
+ * Holder class for Node related
+ */
+public class NodeInfo

Review Comment:
   I am +0 on this change, there's nothing that we leverage from the enum effectively 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307687594


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";
+        private final Set<String> joiningNodes;
+        private final GossipInfoResponse gossipInfo;
+
+        StateWithReplacement(List<String> joiningNodes, List<String> leavingNodes, List<String> movingNodes,
+                             GossipInfoResponse gossipInfo)
+        {
+            super(joiningNodes, leavingNodes, movingNodes);
+            this.joiningNodes = new HashSet<>(joiningNodes);
+            this.gossipInfo = gossipInfo;
+        }
+
+        /**
+         * This method returns state of a node and accounts for a new 'Replacing' state if the node is
+         * replacing a dead node. For returning this state, the method checks status of the node in gossip
+         * information.
+         *
+         * @param endpoint node information represented usually in form of 'ip:port'
+         * @return Node status
+         */
+        @Override
+        String of(String endpoint)
+        {
+            if (joiningNodes.contains(endpoint))
+            {
+                GossipInfoResponse.GossipInfo gossipInfoEntry = gossipInfo.get(endpoint);
+
+                if (gossipInfoEntry != null)
+                {
+                    LOGGER.info("Found gossipInfoEntry={}", gossipInfoEntry);

Review Comment:
   Switching to debug log(per the above comment), as we don't really need this logged at info level. 



##########
build.gradle:
##########
@@ -326,6 +326,8 @@ tasks.register("integrationTest", Test) {
     useJUnitPlatform() {
         includeTags "integrationTest"
     }
+// Uncomment below to run unit tests in parallel
+//    maxParallelForks = Runtime.runtime.availableProcessors() * 2

Review Comment:
   Parallel runs, when enabled do not seem to play well with in-jvm dtests. Since this was introduced in one of the prior commits in this PR, I will remove these commented lines for now. We can re-introduce it as an optimization.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313177974


##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());
+        assertThat(writeRanges).containsExactlyElementsOf(expectedRanges);
+
+        //Sorted and Overlap check
+        validateOrderAndOverlaps(writeRanges);
+        validateOrderAndOverlaps(readRanges);
+    }
+
+    private void validateOrderAndOverlaps(List<Range<BigInteger>> ranges)
+    {
+        for (int r = 0; r < ranges.size() - 1; r++)
+        {
+            assertThat(ranges.get(r).upperEndpoint()).isLessThan(ranges.get(r + 1).upperEndpoint());
+            assertThat(ranges.get(r).intersection(ranges.get(r + 1)).isEmpty()).isTrue();
+        }
+    }
+
+    protected void validateNodeStates(TokenRangeReplicasResponse mappingResponse,
+                                      Set<String> dcReplication,
+                                      Function<Integer, String> statusFunction)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int expectedReplicas = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * dcReplication.size();
+
+        AbstractCassandraTestContext cassandraTestContext = sidecarTestContext.cassandraTestContext();
+        assertThat(mappingResponse.replicaState().size()).isEqualTo(expectedReplicas);
+        for (int i = 1; i <= cassandraTestContext.cluster().size(); i++)
+        {
+            IInstanceConfig config = cassandraTestContext.cluster().get(i).config();
+
+            if (dcReplication.contains(config.localDatacenter()))
+            {
+                String ipAndPort = config.broadcastAddress().getAddress().getHostAddress() + ":"
+                                   + config.broadcastAddress().getPort();
+
+                String expectedStatus = statusFunction.apply(i);
+                assertThat(mappingResponse.replicaState().get(ipAndPort)).isEqualTo(expectedStatus);
+            }
+        }
+    }
+
+    protected UpgradeableCluster getMultiDCCluster(int numNodes,
+                                                   int numDcs,
+                                                   BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws IOException
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier =
+        MultiDcTokenSupplier.evenlyDistributedTokens(numNodes,
+                                                     numDcs,
+                                                     1);
+
+        int totalNodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return cassandraTestContext.configureAndStartCluster(
+        builder -> {
+            builder.withInstanceInitializer(initializer);
+            builder.withTokenSupplier(mdcTokenSupplier);
+            builder.withNodeIdTopology(networkTopology(totalNodeCount,
+                                                       (nodeId) -> nodeId % 2 != 0 ?
+                                                                   dcAndRack("datacenter1", "rack1") :
+                                                                   dcAndRack("datacenter2", "rack2")));
+        });
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return generateExpectedRanges(nodeCount);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(int nodeCount)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = (annotation.numDcs() > 1) ?
+                                      MultiDcTokenSupplier.evenlyDistributedTokens(
+                                      annotation.nodesPerDc() + annotation.newNodesPerDc(),
+                                      annotation.numDcs(),
+                                      1) :
+                                      TokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc() +
+                                                                            annotation.newNodesPerDc(),
+                                                                            1);
+
+        List<Range<BigInteger>> expectedRanges = new ArrayList<>();
+        BigInteger startToken = Partitioner.Murmur3.minToken;
+        BigInteger endToken = Partitioner.Murmur3.maxToken;
+        int node = 1;
+        BigInteger prevToken = new BigInteger(tokenSupplier.tokens(node++).stream().findFirst().get());
+        Range<BigInteger> firstRange = Range.openClosed(startToken, prevToken);
+        expectedRanges.add(firstRange);
+        while (node <= nodeCount)
+        {
+            BigInteger currentToken = new BigInteger(tokenSupplier.tokens(node).stream().findFirst().get());
+            expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            prevToken = currentToken;
+            node++;
+        }
+        expectedRanges.add(Range.openClosed(prevToken, endToken));
+        return expectedRanges;
+    }
+
+    protected Set<String> instancesFromReplicaSet(List<TokenRangeReplicasResponse.ReplicaInfo> replicas)
+    {
+        return replicas.stream()
+                       .flatMap(r -> r.replicasByDatacenter().values().stream())
+                       .flatMap(Collection::stream)
+                       .collect(Collectors.toSet());
+    }
+
+    protected void validateWriteReplicaMappings(List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas,
+                                              Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMapping)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        assertThat(writeReplicas).hasSize(expectedRangeMapping.get("datacenter1").size());

Review Comment:
   Having trouble wrapping my head around why writeReplicas would always be the size of only the first dc here - can you clarify (maybe add a comment) as to why this is the case?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313367893


##########
src/test/integration/org/apache/cassandra/sidecar/routes/TokenRangeIntegrationMultiDCBasicTest.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.

Review Comment:
   NIT: `with cassandra container` isn't true any more - we should replace this comment everywhere it appears with `with the in-jvm dtest framework`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1300514341


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token
+     *
+     * @param other the next range in the range list to compare
+     * @return true if "this" range is larger than the other
+     */
+    protected boolean isLarger(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return this.end.subtract(this.start).compareTo(other.end.subtract(other.start)) > 0;
+    }
+
+    /**
+     * Determines intersection if the next range starts before the current range ends. This method assumes that
+     * the provided ranges are sorted and unwrapped.
+     * When the current range goes all the way to the end, we determine intersection if the next range starts
+     * after the current since all subsequent ranges have to be subsets.
+     *
+     * @param other the range we are currently processing to check if "this" intersects it
+     * @return true if "this" range intersects the other
+     */
+    protected boolean intersects(TokenRangeReplicas other)
+    {
+        if (this.compareTo(other) > 0)
+            throw new IllegalStateException(
+            String.format("Token ranges - (this:%s other:%s) are not ordered", this, other));
+
+        return this.end.compareTo(other.start) > 0 && this.start.compareTo(other.end) < 0; // Start exclusive (DONE)
+    }
+
+    /**
+     * Unwraps the token range if it wraps-around to end either on or after the least token by overriding such
+     * ranges to end at the partitioner max-token value in the former case and splitting into 2 ranges in the latter
+     * case.
+     *
+     * @return list of split ranges
+     */
+    private static List<TokenRangeReplicas> unwrapRange(BigInteger start,
+                                                        BigInteger end,
+                                                        Partitioner partitioner,
+                                                        Set<String> replicaSet)
+    {
+
+        // Range ending at minToken is "unwrapped" to end at the maxToken.
+        // Note: These being open-closed ranges, this will result in exclusion of partitioner's minToken from
+        // allocation. This is by-design as it is never assigned to a node in Cassandra:
+        // https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java#L77
+        if (end.compareTo(partitioner.minToken) == 0)
+        {
+            return Collections.singletonList(
+            new TokenRangeReplicas(start, partitioner.maxToken, partitioner, replicaSet));
+        }
+        else if (start.compareTo(partitioner.maxToken) == 0)
+        {
+            return Collections.singletonList(
+            new TokenRangeReplicas(partitioner.minToken, end, partitioner, replicaSet));
+        }
+
+        // Wrap-around range goes beyond at the "min-token" and is therefore split into two.
+        List<TokenRangeReplicas> unwrapped = new ArrayList<>(2);
+        unwrapped.add(new TokenRangeReplicas(start, partitioner.maxToken, partitioner, replicaSet));
+        unwrapped.add(new TokenRangeReplicas(partitioner.minToken, end, partitioner, replicaSet));
+        return unwrapped;
+    }
+
+
+    /**
+     * Given a list of token ranges with replica-sets, normalizes them by unwrapping around the beginning/min
+     * of the range and removing overlaps to return a sorted list of non-overlapping ranges.
+     * <p>
+     * For an overlapping range that is included in both natural and pending ranges, say R_natural and R_pending
+     * (where R_natural == R_pending), the replicas of both R_natural and R_pending should receive writes.
+     * Therefore, the write-replicas of such range is the union of both replica sets.
+     * This method implements the consolidation process.
+     *
+     * @param ranges
+     * @return sorted list of non-overlapping ranges and replica-sets
+     */
+    public static List<TokenRangeReplicas> normalize(List<TokenRangeReplicas> ranges)
+    {
+
+        if (ranges.stream().noneMatch(r -> r.partitioner.minToken.compareTo(r.start()) == 0))
+        {
+            LOGGER.warn("{} based minToken does not exist in the token ranges", Partitioner.class.getName());
+        }
+
+        return deoverlap(ranges);
+    }
+
+    /**
+     * Given a list of unwrapped (around the starting/min value) token ranges and their replica-sets, return list of
+     * ranges with no overlaps. Any impacted range absorbs the replica-sets from the overlapping range.
+     * This is to ensure that we have most coverage while using the replica-sets as write-replicas.
+     * Overlaps are removed by splitting the original range around the overlap boundaries, resulting in sub-ranges
+     * with replicas from all the overlapping replicas.
+     *
+     *
+     * <pre>
+     * Illustration:
+     * Input with C overlapping with A and B
+     *   |----------A-----------||----------B-------------|
+     *                  |--------C----------|
+     *
+     * Split result: C is split first which further splits A and B to create
+     *  |-----------A----------||----------B-------------|
+     *                 |---C---|----C'----|
+     *
+     * Subsets C & C' are merged into supersets A and B by splitting them. Replica-sets for A,C and B,C are merged
+     * for the resulting ranges.
+     *  |-----A------|----AC---||---BC-----|-----B------|
+     *
+     *  </pre>
+     */
+    private static List<TokenRangeReplicas> deoverlap(List<TokenRangeReplicas> allRanges)
+    {
+        if (allRanges.isEmpty())
+            return allRanges;
+
+        LOGGER.debug("Token ranges to be normalized: {}", allRanges);
+        List<TokenRangeReplicas> ranges = mergeIdenticalRanges(allRanges);
+
+        List<TokenRangeReplicas> output = new ArrayList<>();
+        Iterator<TokenRangeReplicas> iter = ranges.iterator();
+        TokenRangeReplicas current = iter.next();
+
+        while (iter.hasNext())
+        {
+            TokenRangeReplicas next = iter.next();
+            if (!current.intersects(next))
+            {
+                output.add(current);
+                current = next;
+            }
+            else
+            {
+                current = processIntersectingRanges(output, iter, current, next);
+            }
+        }
+        if (current != null)
+            output.add(current);
+        return output;
+    }
+
+    private static List<TokenRangeReplicas> mergeIdenticalRanges(List<TokenRangeReplicas> ranges)
+    {
+        Map<TokenRangeReplicas, Set<String>> rangeMapping = new HashMap<>();
+        for (TokenRangeReplicas r: ranges)
+        {
+            if (!rangeMapping.containsKey(r))
+            {
+                rangeMapping.put(r, r.replicaSet);
+            }
+            else
+            {
+                rangeMapping.get(r).addAll(r.replicaSet);
+            }
+        }
+
+        List<TokenRangeReplicas> merged = new ArrayList<>();
+        for (Map.Entry<TokenRangeReplicas, Set<String>> entry : rangeMapping.entrySet())
+        {
+            TokenRangeReplicas r = entry.getKey();
+            if (!r.replicaSet().equals(entry.getValue()))
+            {
+                r.replicaSet().addAll(entry.getValue());
+            }
+            merged.add(r);
+        }
+        Collections.sort(merged);
+        return merged;
+    }
+
+    /**
+     * Splits intersecting token ranges starting from the provided cursors and the iterator, while accumulating
+     * overlapping replicas into each sub-range.
+     * <p>
+     * The algorithm 1) extracts all intersecting ranges at the provided cursor, and 2) Maintains a min-heap of all
+     * intersecting ranges ordered by the end of the range, so that the least common sub-range relative to the current
+     * range can be extracted.
+     *
+     * @param output  ongoing list of resulting non-overlapping ranges
+     * @param iter    iterator over the list of ranges
+     * @param current cursor to the current, intersecting range
+     * @param next    cursor to the intersecting range after the current range
+     * @return cursor to the subsequent non-intersecting range
+     */
+    protected static TokenRangeReplicas processIntersectingRanges(List<TokenRangeReplicas> output,
+                                                                  Iterator<TokenRangeReplicas> iter,
+                                                                  TokenRangeReplicas current,
+                                                                  TokenRangeReplicas next)
+    {
+        PriorityQueue<TokenRangeReplicas> rangeHeap =
+        new PriorityQueue<>((n1, n2) -> (!n1.end.equals(n2.end())) ?
+                                        n1.end().compareTo(n2.end()) : n1.compareTo(n2));
+
+        List<TokenRangeReplicas> intersectingRanges = new ArrayList<>();
+        next = extractIntersectingRanges(intersectingRanges::add, iter, current, next);
+        rangeHeap.add(intersectingRanges.get(0));
+        intersectingRanges.stream().skip(1).forEach(r -> {
+            if (!rangeHeap.isEmpty())
+            {
+                TokenRangeReplicas range = rangeHeap.peek();
+                // Use the last processed range's end as the new range's start
+                // Except when its the first range, in which case, we use the queue-head's start
+                BigInteger newStart = output.isEmpty() ? range.start() : output.get(output.size() - 1).end();
+
+                if (r.start().compareTo(rangeHeap.peek().end()) == 0)
+                {
+                    output.add(new TokenRangeReplicas(newStart,
+                                                      r.start(),
+                                                      range.partitioner,
+                                                      getBatchReplicas(rangeHeap)));
+                    rangeHeap.poll();
+                }
+                else if (r.start().compareTo(rangeHeap.peek().end()) > 0)
+                {
+                    output.add(new TokenRangeReplicas(newStart,
+                                                      range.end(),
+                                                      range.partitioner,
+                                                      getBatchReplicas(rangeHeap)));
+                    rangeHeap.poll();
+                }
+                // Start-token is before the first intersecting range end. We have not encountered end of the range, so
+                // it is not removed from the heap yet.
+                else
+                {
+                    if (newStart.compareTo(r.start()) != 0)
+                    {
+                        output.add(new TokenRangeReplicas(newStart,
+                                                          r.start(),
+                                                          range.partitioner,
+                                                          getBatchReplicas(rangeHeap)));
+                    }
+                }
+                rangeHeap.add(r);
+            }
+        });
+
+        // Remaining intersecting ranges from heap are processed
+        while (!rangeHeap.isEmpty())
+        {
+            LOGGER.info("Non-empty queue:" + rangeHeap.size());
+            TokenRangeReplicas nextVal = rangeHeap.peek();
+            BigInteger newStart = output.isEmpty() ? nextVal.start() : output.get(output.size() - 1).end();
+            // Corner case w/ common end ranges - we do not add redundant single token range
+            if (newStart.compareTo(nextVal.end()) != 0)
+            {
+                output.add(new TokenRangeReplicas(newStart,
+                                                  nextVal.end(),
+                                                  nextVal.partitioner,
+                                                  getBatchReplicas(rangeHeap)));
+            }
+            rangeHeap.poll();
+        }
+        return next;
+    }
+
+    /**
+     * Extract all the intersecting ranges starting from the current cursor, which we know is intersecting with the
+     * next range. Note that the cursor is moved forward until a non-intersecting range is found.
+     *
+     * @param rangeConsumer functional interface to collect candidate intersecting ranges
+     * @param iter          ongoing iterator over the entire range-set
+     * @param current       cursor to the current, intersecting range
+     * @param next          cursor to the next intersecting range
+     * @return list of intersecting ranges starting at the specified cursor
+     */
+    private static TokenRangeReplicas extractIntersectingRanges(Consumer<TokenRangeReplicas> rangeConsumer,
+                                                                Iterator<TokenRangeReplicas> iter,
+                                                                TokenRangeReplicas current,
+                                                                TokenRangeReplicas next)
+    {
+        // we know that current and next intersect
+        rangeConsumer.accept(current);
+        rangeConsumer.accept(next);
+        current = (current.contains(next)) ? current : next;
+        next = null;
+        while (iter.hasNext())
+        {
+            next = iter.next();
+            if (!current.intersects(next))
+            {
+                break;
+            }
+            rangeConsumer.accept(next);
+            // when next is subset of current, we keep tracking current
+            current = (current.contains(next)) ? current : next;
+            next = null;
+        }
+        return next;
+    }
+
+    // TODO: Verify why we need all replicas from queue

Review Comment:
   Resolved. Will remove



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1300510095


##########
src/test/integration/org/apache/cassandra/testing/TestVersionSupplier.java:
##########
@@ -35,7 +35,7 @@ public class TestVersionSupplier
     Stream<TestVersion> testVersions()
     {
         // By default, we test 2 versions that will exercise oldest and newest supported versions
-        String versions = System.getProperty("cassandra.sidecar.versions_to_test", "4.0,5.0");
+        String versions = System.getProperty("cassandra.sidecar.versions_to_test", "5.0");

Review Comment:
   Currently validating tests with both versions enabled. Will revert this change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308790925


##########
src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java:
##########
@@ -148,34 +149,21 @@ private BeforeEachCallback beforeEach()
                 int nodesPerDc = annotation.nodesPerDc();
                 int dcCount = annotation.numDcs();
                 int newNodesPerDc = annotation.newNodesPerDc(); // if the test wants to add more nodes later
+                Preconditions.checkArgument(newNodesPerDc >= 0,
+                                            "newNodesPerDc cannot be a negative number");
+                int originalNodeCount = nodesPerDc * dcCount;
                 int finalNodeCount = dcCount * (nodesPerDc + newNodesPerDc);
                 Versions.Version requestedVersion = versions.getLatest(new Semver(version.version(),
                                                                                   Semver.SemverType.LOOSE));
                 SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(version.version());
 
                 UpgradeableCluster.Builder clusterBuilder =
-                    UpgradeableCluster.build(nodesPerDc * dcCount)
-                                      .withVersion(requestedVersion)
-                                      .withDCs(dcCount)
-                                      .withDataDirCount(annotation.numDataDirsPerInstance())
-                                      .withConfig(config -> {
-                                      if (annotation.nativeTransport())
-                                      {
-                                          config.with(Feature.NATIVE_PROTOCOL);
-                                      }
-                                      if (annotation.jmx())
-                                      {
-                                          config.with(Feature.JMX);
-                                      }
-                                      if (annotation.gossip())
-                                      {
-                                          config.with(Feature.GOSSIP);
-                                      }
-                                      if (annotation.network())
-                                      {
-                                          config.with(Feature.NETWORK);
-                                      }
-                                  });
+                UpgradeableCluster.build(originalNodeCount)
+//                                  .withDynamicPortAllocation(true) // to allow parallel test runs

Review Comment:
   we should be able to leverage this feature now. https://issues.apache.org/jira/browse/CASSANDRA-18722 has been merged



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1309080569


##########
src/test/resources/logback-test.xml:
##########
@@ -20,7 +20,7 @@
 
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
     <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
-      <level>INFO</level>
+      <level>DEBUG</level>

Review Comment:
   I don't have a strong opinion on this, can revert since these are tests.
   
   @yifan-c - Does this make sense, since you recommended this change?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308147114


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());

Review Comment:
   ~The formatting was not done yet~
   
   GH page displayed stale code. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] yifan-c commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "yifan-c (via GitHub)" <gi...@apache.org>.
yifan-c commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308147114


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());

Review Comment:
   The formatting was not done yet



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308799235


##########
src/test/resources/logback-test.xml:
##########
@@ -20,7 +20,7 @@
 
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
     <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
-      <level>INFO</level>
+      <level>DEBUG</level>

Review Comment:
   I think INFO is fine, the tests already print way too much output



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312439641


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token
+     *
+     * @param other the next range in the range list to compare
+     * @return true if "this" range is larger than the other
+     */
+    protected boolean isLarger(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return this.end.subtract(this.start).compareTo(other.end.subtract(other.start)) > 0;
+    }
+
+    /**
+     * Determines intersection if the next range starts before the current range ends. This method assumes that
+     * the provided ranges are sorted and unwrapped.
+     * When the current range goes all the way to the end, we determine intersection if the next range starts
+     * after the current since all subsequent ranges have to be subsets.
+     *
+     * @param other the range we are currently processing to check if "this" intersects it
+     * @return true if "this" range intersects the other
+     */
+    protected boolean intersects(TokenRangeReplicas other)
+    {
+        if (this.compareTo(other) > 0)
+            throw new IllegalStateException(
+            String.format("Token ranges - (this:%s other:%s) are not ordered", this, other));
+
+        return this.end.compareTo(other.start) > 0 && this.start.compareTo(other.end) < 0; // Start exclusive (DONE)
+    }

Review Comment:
   Apart from here there are other operations that rely on the ranges being processed to be sorted and breaks if the prerequisite is not met. Although that should "never" happen since we are explicitly sorting and unwrapping upfront. 
   
   Like you mentioned earlier, I'm leaning towards performing the assertions on the assumptions up-front, if for any reason the ranges are not what we expect them to be.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#issuecomment-1613749152

   > PR looks good in general, but would it make sense to add the endpoint to the client so it can be consumed by third-party libraries?
   
   Yep, added to client in the next commit.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307688445


##########
src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.net.UnknownHostException;
+
+import org.apache.commons.lang3.StringUtils;
+
+import com.datastax.driver.core.Metadata;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.handler.HttpException;
+import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasRequest;
+import org.apache.cassandra.sidecar.common.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static org.apache.cassandra.sidecar.utils.HttpExceptions.cassandraServiceUnavailable;
+
+/**
+ * Handler which provides token range to read and write replica mapping
+ *
+ * <p>This handler provides token range replicas along with the state of the replicas. For the purpose
+ * of identifying the state of a newly joining node to replace a dead node from a newly joining node,
+ * a new state 'Replacing' has been added.
+ * It is represented by
+ * {@code org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicaProvider.StateWithReplacement}
+ */
+@Singleton
+public class TokenRangeReplicaMapHandler extends AbstractHandler<TokenRangeReplicasRequest>
+{
+
+    @Inject
+    public TokenRangeReplicaMapHandler(InstanceMetadataFetcher metadataFetcher,
+                                       CassandraInputValidator validator,
+                                       ExecutorPools executorPools)
+    {
+        super(metadataFetcher, executorPools, validator);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void handleInternal(RoutingContext context,
+                               HttpServerRequest httpRequest,
+                               String host,
+                               SocketAddress remoteAddress,
+                               TokenRangeReplicasRequest request)
+    {
+        CassandraAdapterDelegate delegate = metadataFetcher.delegate(host);
+
+        StorageOperations storageOperations = delegate.storageOperations();
+        Metadata metadata = delegate.metadata();
+        if (storageOperations == null || metadata == null)
+        {
+            context.fail(cassandraServiceUnavailable());
+            return;
+        }
+
+        executorPools.service().executeBlocking(promise -> {
+            try
+            {
+                context.json(storageOperations.tokenRangeReplicas(request.keyspace(), metadata.getPartitioner()));
+            }
+            catch (UnknownHostException e)
+            {
+                processFailure(e, context, host, remoteAddress, request);

Review Comment:
   Will update.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307689009


##########
src/test/integration/org/apache/cassandra/sidecar/routes/MultiDcTokenSupplier.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.distributed.api.TokenSupplier;
+
+/**
+ * Static factory holder that provides a multi-DC token supplier
+ */
+public class MultiDcTokenSupplier
+{
+
+    static TokenSupplier evenlyDistributedTokens(int numNodes, int numDcs, int numTokens)

Review Comment:
   So, we perform `numNodes * numDcs * numTokensPerNode` to compute the total no. tokens across all DCs and nodes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1309244264


##########
common/build.gradle:
##########
@@ -41,6 +41,8 @@ repositories {
 
 test {
     useJUnitPlatform()
+// Uncomment below to run unit tests in parallel
+//    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1

Review Comment:
   @frankgh - Can you comment on @yifan-c 's question, since this was introduced in your change? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1310535055


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     naturalReplicaMappings);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        LOGGER.debug("Pending token range mappings for keyspace={}, pendingRangeMappings={}",
+                     keyspace,
+                     pendingRangeMappings);
+        List<TokenRangeReplicas> naturalTokenRangeReplicas = transformRangeMappings(naturalReplicaMappings,
+                                                                                    partitioner);
+        List<TokenRangeReplicas> pendingTokenRangeReplicas = transformRangeMappings(pendingRangeMappings,
+                                                                                    partitioner);
+
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> allTokenRangeReplicas = new ArrayList<>(naturalTokenRangeReplicas);
+        allTokenRangeReplicas.addAll(pendingTokenRangeReplicas);
+
+        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(allTokenRangeReplicas);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<ReplicaInfo> writeReplicas = writeReplicasFromPendingRanges(allTokenRangeReplicas, hostToDatacenter);
+
+        List<ReplicaInfo> readReplicas = readReplicasFromReplicaMapping(naturalTokenRangeReplicas, hostToDatacenter);
+        Map<String, String> replicaToStateMap = replicaToStateMap(allTokenRangeReplicas, storage);
+
+        return new TokenRangeReplicasResponse(replicaToStateMap,
+                                              writeReplicas,
+                                              readReplicas);
+    }
+
+    private Map<String, String> replicaToStateMap(List<TokenRangeReplicas> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .map(TokenRangeReplicas::replicaSet)
+                         .flatMap(Collection::stream)
+                         .distinct()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    protected EndpointSnitchJmxOperations initializeEndpointProxy()
+    {
+        return jmxClient.proxy(EndpointSnitchJmxOperations.class, ENDPOINT_SNITCH_INFO_OBJ_NAME);
+    }
+
+    protected StorageJmxOperations initializeStorageOps()
+    {
+        return jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+    }
+
+
+    protected String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<ReplicaInfo> writeReplicasFromPendingRanges(List<TokenRangeReplicas> tokenRangeReplicaSet,
+                                                             Map<String, String> hostToDatacenter)
+    {
+//        Map<String, String> hostToDatacenter = buildHostToDatacenterMapping(tokenRangeReplicaSet);
+        // Candidate write-replica mappings are normalized by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(tokenRangeReplicaSet).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new ReplicaInfo(range.start().toString(),
+                                                            range.end().toString(),
+                                                            replicasByDc);
+                                 })
+                                 .collect(toList());
+    }
+
+    private List<TokenRangeReplicas> transformRangeMappings(Map<List<String>, List<String>> replicaMappings,
+                                                            Partitioner partitioner)
+    {
+        return replicaMappings.entrySet()
+                              .stream()
+                              .map(entry -> generateTokenRangeReplicas(new BigInteger(entry.getKey().get(0)),
+                                                                       new BigInteger(entry.getKey().get(1)),
+                                                                       partitioner,
+                                                                       new HashSet<>(entry.getValue())))
+                              .flatMap(Collection::stream)
+                              .collect(toList());
+    }
+
+
+    private List<ReplicaInfo> readReplicasFromReplicaMapping(List<TokenRangeReplicas> naturalTokenRangeReplicas,
+                                                             Map<String, String> hostToDatacenter)

Review Comment:
   Addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312444885


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token
+     *
+     * @param other the next range in the range list to compare
+     * @return true if "this" range is larger than the other
+     */
+    protected boolean isLarger(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return this.end.subtract(this.start).compareTo(other.end.subtract(other.start)) > 0;
+    }
+
+    /**
+     * Determines intersection if the next range starts before the current range ends. This method assumes that
+     * the provided ranges are sorted and unwrapped.
+     * When the current range goes all the way to the end, we determine intersection if the next range starts
+     * after the current since all subsequent ranges have to be subsets.
+     *
+     * @param other the range we are currently processing to check if "this" intersects it
+     * @return true if "this" range intersects the other
+     */
+    protected boolean intersects(TokenRangeReplicas other)
+    {
+        if (this.compareTo(other) > 0)
+            throw new IllegalStateException(
+            String.format("Token ranges - (this:%s other:%s) are not ordered", this, other));
+
+        return this.end.compareTo(other.start) > 0 && this.start.compareTo(other.end) < 0; // Start exclusive (DONE)
+    }

Review Comment:
   Another option would be to perform the assertion within `processIntersectingRanges` instead and do what you're suggesting in the `intersects()` method. That way, we only fail in the event there are intersections (and the ranges are out of order). 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312460343


##########
adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java:
##########
@@ -0,0 +1,824 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    private boolean hasOverlaps(List<TokenRangeReplicas> rangeList)
+    {
+        Collections.sort(rangeList);
+        for (int c = 0, i = 1; i < rangeList.size(); i++)
+        {
+            if (rangeList.get(c++).end().compareTo(rangeList.get(i).start()) > 0) return true;
+        }
+        return false;
+    }
+
+    private boolean checkContains(List<TokenRangeReplicas> resultList, TokenRangeReplicas expected)
+    {
+        return resultList.stream()
+                         .map(TokenRangeReplicas::toString)
+                         .anyMatch(r -> r.equals(expected.toString()));
+    }

Review Comment:
   Yep agree, I'd blame IntelliJ's insertion ordering :)  Addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312463547


##########
adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java:
##########
@@ -0,0 +1,824 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    private boolean hasOverlaps(List<TokenRangeReplicas> rangeList)
+    {
+        Collections.sort(rangeList);
+        for (int c = 0, i = 1; i < rangeList.size(); i++)
+        {
+            if (rangeList.get(c++).end().compareTo(rangeList.get(i).start()) > 0) return true;
+        }
+        return false;
+    }
+
+    private boolean checkContains(List<TokenRangeReplicas> resultList, TokenRangeReplicas expected)
+    {
+        return resultList.stream()
+                         .map(TokenRangeReplicas::toString)
+                         .anyMatch(r -> r.equals(expected.toString()));
+    }
+
+    // non-overlapping ranges
+    @Test
+    public void simpleTest()
+    {
+        List<TokenRangeReplicas> simpleList = createSimpleTokenRangeReplicaList();
+        LOGGER.info("Input:" + simpleList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(simpleList);

Review Comment:
   Yep, will add validation for input/output lists match.
   
   The intention behind the current check is that `normalize` does not do anything bad when there are no overlaps as the simplest case of the test, with other test cases covering the overlapping/sub-set etc. scenarios.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313399299


##########
src/test/integration/org/apache/cassandra/sidecar/routes/TokenRangeIntegrationLeavingTest.java:
##########
@@ -0,0 +1,786 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import net.bytebuddy.ByteBuddy;
+import net.bytebuddy.description.type.TypeDescription;
+import net.bytebuddy.dynamic.ClassFileLocator;
+import net.bytebuddy.dynamic.TypeResolutionStrategy;
+import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
+import net.bytebuddy.implementation.MethodDelegation;
+import net.bytebuddy.implementation.bind.annotation.SuperCall;
+import net.bytebuddy.pool.TypePool;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.distributed.shared.ClusterUtils;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import org.apache.cassandra.utils.Shared;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Cluster shrink scenarios integration tests for token range replica mapping endpoint with cassandra container.
+ */
+@ExtendWith(VertxExtension.class)
+public class TokenRangeIntegrationLeavingTest extends BaseTokenRangeIntegrationTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithKeyspaceLeavingNode(VertxTestContext context,
+                                                ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        runLeavingTestScenario(context,
+                               cassandraTestContext,
+                               1,
+                               BBHelperSingleLeavingNode::install,
+                               BBHelperSingleLeavingNode.TRANSIENT_STATE_START,
+                               BBHelperSingleLeavingNode.TRANSIENT_STATE_END,
+                               generateExpectedRangeMappingSingleLeavingNode());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithMultipleLeavingNodes(VertxTestContext context,
+                                                 ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        runLeavingTestScenario(context,
+                               cassandraTestContext,
+                               2,
+                               BBHelperMultipleLeavingNodes::install,
+                               BBHelperMultipleLeavingNodes.TRANSIENT_STATE_START,
+                               BBHelperMultipleLeavingNodes.TRANSIENT_STATE_END,
+                               generateExpectedRangeMappingMultipleLeavingNodes());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 6, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingHalveClusterSize(VertxTestContext context,
+                                         ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        runLeavingTestScenario(context,
+                               cassandraTestContext,
+                               3,
+                               BBHelperHalveClusterSize::install,
+                               BBHelperHalveClusterSize.TRANSIENT_STATE_START,
+                               BBHelperHalveClusterSize.TRANSIENT_STATE_END,
+                               generateExpectedRangeMappingHalveClusterSize());
+    }
+
+    @CassandraIntegrationTest(
+    nodesPerDc = 5, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithLeavingNodesMultiDC(VertxTestContext context,
+                                                ConfigurableCassandraTestContext cassandraTestContext)
+    throws Exception
+    {
+
+        int leavingNodesPerDC = 1;
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int numNodes = annotation.nodesPerDc() + annotation.newNodesPerDc();
+        UpgradeableCluster cluster = getMultiDCCluster(numNodes,
+                                                       annotation.numDcs(),
+                                                       BBHelperLeavingNodesMultiDC::install,
+                                                       cassandraTestContext);
+
+        runLeavingTestScenario(context,
+                               leavingNodesPerDC,
+                               BBHelperLeavingNodesMultiDC.TRANSIENT_STATE_START,
+                               BBHelperLeavingNodesMultiDC.TRANSIENT_STATE_END,
+                               cluster,
+                               generateExpectedRangeMappingLeavingNodeMultiDC());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 6, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingMultiDCHalveClusterSize(VertxTestContext context,
+                                                ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+
+        int leavingNodesPerDC = 3;
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int numNodes = annotation.nodesPerDc() + annotation.newNodesPerDc();
+        UpgradeableCluster cluster = getMultiDCCluster(numNodes,
+                                                       annotation.numDcs(),
+                                                       BBHelperHalveClusterMultiDC::install,
+                                                       cassandraTestContext);
+
+        Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings
+        = generateExpectedRangeHalveClusterSizeMultiDC();
+        runLeavingTestScenario(context,
+                               leavingNodesPerDC,
+                               BBHelperHalveClusterMultiDC.TRANSIENT_STATE_START,
+                               BBHelperHalveClusterMultiDC.TRANSIENT_STATE_END,
+                               cluster,
+                               expectedRangeMappings);
+    }
+
+    void runLeavingTestScenario(VertxTestContext context,
+                                ConfigurableCassandraTestContext cassandraTestContext,
+                                int leavingNodesPerDC,
+                                BiConsumer<ClassLoader, Integer> instanceInitializer,
+                                CountDownLatch transientStateStart,
+                                CountDownLatch transientStateEnd,
+                                Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    throws Exception
+    {
+
+        UpgradeableCluster cluster = cassandraTestContext.configureAndStartCluster(
+        builder -> builder.withInstanceInitializer(instanceInitializer));
+
+        runLeavingTestScenario(context,
+                               leavingNodesPerDC,
+                               transientStateStart,
+                               transientStateEnd,
+                               cluster,
+                               expectedRangeMappings);
+    }
+
+    void runLeavingTestScenario(VertxTestContext context,
+                                int leavingNodesPerDC,
+                                CountDownLatch transientStateStart,
+                                CountDownLatch transientStateEnd,
+                                UpgradeableCluster cluster,
+                                Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    throws Exception
+    {
+        try
+        {
+            CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+            Set<String> dcReplication;
+
+            if (annotation.numDcs() > 1)

Review Comment:
   This logic is repeated a bunch of times - maybe move to the integration test base?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1300509025


##########
README.md:
##########
@@ -49,6 +49,19 @@ The build script supports two parameters:
 Remove any versions you may not want to test with. We recommend at least the latest (released) 4.X series and `trunk`.
 See Testing for more details on how to choose which Cassandra versions to use while testing.
 
+For multi-node in-jvm dtests, network aliases will need to be setup for each Cassandra node. The tests assume each node's 
+ip address is 127.0.0.x, where x is the node id. 
+
+For example if you populated your cluster with 3 nodes, create interfaces for 127.0.0.2 and 127.0.0.3 (the first node of course uses 127.0.0.1).
+
+### macOS network aliases
+To get up and running, create a temporary alias for every node except the first:
+
+```
+sudo ifconfig lo0 alias 127.0.0.2

Review Comment:
   I think we have up to 10 for the existing tests, should we just tell devs to do this:
   ```suggestion
    for i in {2..10}; do sudo ifconfig lo0 alias "127.0.0.${i}"; done
   ```



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/Partitioner.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+
+/**
+ * Represents types of Partitioners supported and the corresponding starting token values
+ */
+public enum Partitioner
+{
+    Murmur3(BigInteger.valueOf(Long.MIN_VALUE), BigInteger.valueOf(Long.MAX_VALUE)),
+    Random(BigInteger.ONE.negate(), BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE));

Review Comment:
   so looking at the Cassandra code, for Murmur3 the maximum is Long.MAX_VALUE (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java#L51). but for Random in Cassandra the maximum is `2^127` (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java#L53) , however here we declare it as `(2^127) - 1`. Any reason we subtract one for random here?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);

Review Comment:
   Refer to `RingProvider`, we need to provide an initialization method in case an extension class wishes to reuse the code with a custom StorageJmxOperation interface (i.e. Cassandra 3.11 implementation)
   ```suggestion
           StorageJmxOperations storage = initializeStorageOps();
   ```



##########
src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.net.UnknownHostException;
+
+import org.apache.commons.lang3.StringUtils;
+
+import com.datastax.driver.core.Metadata;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.handler.HttpException;
+import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasRequest;
+import org.apache.cassandra.sidecar.common.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static org.apache.cassandra.sidecar.utils.HttpExceptions.cassandraServiceUnavailable;
+
+/**
+ * Handler which provides token range to read and write replica mapping
+ *
+ * <p>This handler provides token range replicas along with the state of the replicas. For the purpose
+ * of identifying the state of a newly joining node to replace a dead node from a newly joining node,
+ * a new state 'Replacing' has been added.
+ * It is represented by
+ * {@code org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicaProvider.StateWithReplacement}
+ */
+@Singleton
+public class TokenRangeReplicaMapHandler extends AbstractHandler<TokenRangeReplicasRequest>
+{
+
+    @Inject
+    public TokenRangeReplicaMapHandler(InstanceMetadataFetcher metadataFetcher,
+                                       CassandraInputValidator validator,
+                                       ExecutorPools executorPools)
+    {
+        super(metadataFetcher, executorPools, validator);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void handleInternal(RoutingContext context,
+                               HttpServerRequest httpRequest,
+                               String host,
+                               SocketAddress remoteAddress,
+                               TokenRangeReplicasRequest request)
+    {
+        CassandraAdapterDelegate delegate = metadataFetcher.delegate(host);
+
+        StorageOperations storageOperations = delegate.storageOperations();
+        Metadata metadata = delegate.metadata();
+        if (storageOperations == null || metadata == null)
+        {
+            context.fail(cassandraServiceUnavailable());
+            return;
+        }
+
+        executorPools.service().executeBlocking(promise -> {
+            try
+            {
+                context.json(storageOperations.tokenRangeReplicas(request.keyspace(), metadata.getPartitioner()));
+            }
+            catch (UnknownHostException e)
+            {
+                processFailure(e, context, host, remoteAddress, request);
+            }
+        }).onFailure(cause -> processFailure(cause, context, host, remoteAddress, request));
+    }
+
+    @Override
+    protected TokenRangeReplicasRequest extractParamsOrThrow(RoutingContext context)
+    {
+        return new TokenRangeReplicasRequest(keyspace(context, true));
+    }
+
+    @Override
+    protected void processFailure(Throwable cause, RoutingContext context, String host, SocketAddress remoteAddress,
+                                  TokenRangeReplicasRequest request)
+    {
+        if (cause instanceof AssertionError &&
+            StringUtils.contains(cause.getMessage(), "Unknown keyspace"))
+        {
+            context.fail(new HttpException(HttpResponseStatus.NOT_FOUND.code(), cause.getMessage()));

Review Comment:
   nit, since we have these helper methods:
   ```suggestion
               context.fail(HttpExceptions.wrapHttpException(HttpResponseStatus.NOT_FOUND, cause.getMessage()));
   ```
   



##########
src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.net.UnknownHostException;
+
+import org.apache.commons.lang3.StringUtils;
+
+import com.datastax.driver.core.Metadata;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import io.vertx.ext.web.handler.HttpException;
+import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasRequest;
+import org.apache.cassandra.sidecar.common.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static org.apache.cassandra.sidecar.utils.HttpExceptions.cassandraServiceUnavailable;
+
+/**
+ * Handler which provides token range to read and write replica mapping
+ *
+ * <p>This handler provides token range replicas along with the state of the replicas. For the purpose
+ * of identifying the state of a newly joining node to replace a dead node from a newly joining node,
+ * a new state 'Replacing' has been added.
+ * It is represented by
+ * {@code org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicaProvider.StateWithReplacement}
+ */
+@Singleton
+public class TokenRangeReplicaMapHandler extends AbstractHandler<TokenRangeReplicasRequest>
+{
+
+    @Inject
+    public TokenRangeReplicaMapHandler(InstanceMetadataFetcher metadataFetcher,
+                                       CassandraInputValidator validator,
+                                       ExecutorPools executorPools)
+    {
+        super(metadataFetcher, executorPools, validator);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void handleInternal(RoutingContext context,
+                               HttpServerRequest httpRequest,
+                               String host,
+                               SocketAddress remoteAddress,
+                               TokenRangeReplicasRequest request)
+    {
+        CassandraAdapterDelegate delegate = metadataFetcher.delegate(host);
+
+        StorageOperations storageOperations = delegate.storageOperations();
+        Metadata metadata = delegate.metadata();
+        if (storageOperations == null || metadata == null)
+        {
+            context.fail(cassandraServiceUnavailable());
+            return;
+        }
+
+        executorPools.service().executeBlocking(promise -> {
+            try
+            {
+                context.json(storageOperations.tokenRangeReplicas(request.keyspace(), metadata.getPartitioner()));
+            }
+            catch (UnknownHostException e)
+            {
+                processFailure(e, context, host, remoteAddress, request);

Review Comment:
   it looks like we never throw the `UnknownHostException` exception, we are wrapping it in `RuntimeException`, should we catch that instead here?



##########
client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java:
##########
@@ -162,6 +163,23 @@ public CompletableFuture<TimeSkewResponse> timeSkew(List<? extends SidecarInstan
                                             .build());
     }
 
+    /**
+     * Executes the token-range replicas request using the default retry policy and configured selection policy
+     *
+     * @param instances the list of Sidecar instances to try for this request
+     * @param keyspace  the keyspace in Cassandra
+     * @return a completable future of the token-range replicas
+     */
+    public CompletableFuture<TokenRangeReplicasResponse> tokenRangeReplicas(List<? extends SidecarInstance> instances,
+                                                                            String keyspace)
+    {
+        SidecarInstancesProvider instancesProvider = new SimpleSidecarInstancesProvider(instances);
+        InstanceSelectionPolicy instanceSelectionPolicy = new RandomInstanceSelectionPolicy(instancesProvider);
+        return executeRequestAsync(requestBuilder()
+                                   .instanceSelectionPolicy(instanceSelectionPolicy)
+                                   .tokenRangeReplicasRequest(keyspace).build());

Review Comment:
   minor nit
   ```suggestion
                                      .tokenRangeReplicasRequest(keyspace)
                                      .build());
   ```



##########
common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java:
##########
@@ -240,21 +240,20 @@ public boolean isUp()
 
     public void close()
     {
-        Session activeSession = cqlSessionProvider.localCql();
+        nodeSettings = null;
+        Session activeSession = cqlSessionProvider.close();
         if (activeSession != null)
         {
             maybeUnregisterHostListener(activeSession);
-            cqlSessionProvider.close();
         }
-        nodeSettings = null;
         try
         {
             jmxClient.close();
         }
         catch (IOException e)
         {
             // Can't throw unchecked exceptions here, so wrap and rethrow
-            throw new RuntimeException(e);
+            LOGGER.warn("Unable to close JMX client", e);

Review Comment:
   update comment above, since we are not re-throwing the exception anymore



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);

Review Comment:
   Similar comment
   ```suggestion
           EndpointSnitchJmxOperations endpointSnitchInfo = initializeEndpointProxy();
   ```



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)

Review Comment:
   Similar comment to above, also make sure the initialization methods are protected so they can be overridden 
   ```suggestion
           return initializeClusterMembershipOps()
   ```



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasResponse.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.common.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Class response for the {@link TokenRangeReplicasRequest}
+ */
+public class TokenRangeReplicasResponse
+{
+    private final Map<String, String> replicaState;
+    private final List<ReplicaInfo> writeReplicas;
+    private final List<ReplicaInfo> readReplicas;
+
+    /**
+     * Constructs token range replicas response object with given params.
+     *
+     * @param replicaState  mapping replica to it's state information
+     * @param writeReplicas list of write replicas {@link ReplicaInfo} instances breakdown by token range
+     * @param readReplicas  list of read replica {@link ReplicaInfo} instances breakdown by token range
+     */
+    public TokenRangeReplicasResponse(@JsonProperty("replicaState") Map<String, String> replicaState,
+                                      @JsonProperty("writeReplicas") List<ReplicaInfo> writeReplicas,
+                                      @JsonProperty("readReplicas") List<ReplicaInfo> readReplicas)
+    {
+        this.replicaState = replicaState;
+        this.writeReplicas = writeReplicas;
+        this.readReplicas = readReplicas;
+    }
+
+    /**
+     * @return returns replica to it's state information mapping
+     */
+    @JsonProperty("replicaState")
+    public Map<String, String> replicaState()
+    {
+        return replicaState;
+    }
+
+    /**
+     * @return returns the {@link ReplicaInfo} instances representing write replicas for each token range
+     */
+    @JsonProperty("writeReplicas")
+    public List<ReplicaInfo> writeReplicas()
+    {
+        return writeReplicas;
+    }
+
+    /**
+     * @return returns the {@link ReplicaInfo} instances representing read replicas for each token range

Review Comment:
   ```suggestion
        * @return the {@link ReplicaInfo} instances representing read replicas for each token range
   ```



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/GossipInfoResponse.java:
##########
@@ -53,6 +53,33 @@
  */
 public class GossipInfoResponse extends HashMap<String, GossipInfoResponse.GossipInfo>
 {
+    /**
+     * Overrides the {@link #get(Object)} method. The gossip info keys usually start with the format
+     * {@code /ip:port}. Some clients may be unaware of the preceding {@code slash}, and lookups can

Review Comment:
   this bug is exhibited in analytics library as seen here: https://github.com/apache/cassandra-analytics/blob/trunk/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java#L523



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasResponse.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.common.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Class response for the {@link TokenRangeReplicasRequest}
+ */
+public class TokenRangeReplicasResponse
+{
+    private final Map<String, String> replicaState;
+    private final List<ReplicaInfo> writeReplicas;
+    private final List<ReplicaInfo> readReplicas;
+
+    /**
+     * Constructs token range replicas response object with given params.
+     *
+     * @param replicaState  mapping replica to it's state information
+     * @param writeReplicas list of write replicas {@link ReplicaInfo} instances breakdown by token range
+     * @param readReplicas  list of read replica {@link ReplicaInfo} instances breakdown by token range
+     */
+    public TokenRangeReplicasResponse(@JsonProperty("replicaState") Map<String, String> replicaState,
+                                      @JsonProperty("writeReplicas") List<ReplicaInfo> writeReplicas,
+                                      @JsonProperty("readReplicas") List<ReplicaInfo> readReplicas)
+    {
+        this.replicaState = replicaState;
+        this.writeReplicas = writeReplicas;
+        this.readReplicas = readReplicas;
+    }
+
+    /**
+     * @return returns replica to it's state information mapping

Review Comment:
   ```suggestion
        * @return the replica to state information mapping
   ```



##########
common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasResponse.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.common.data;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Class response for the {@link TokenRangeReplicasRequest}
+ */
+public class TokenRangeReplicasResponse
+{
+    private final Map<String, String> replicaState;
+    private final List<ReplicaInfo> writeReplicas;
+    private final List<ReplicaInfo> readReplicas;
+
+    /**
+     * Constructs token range replicas response object with given params.
+     *
+     * @param replicaState  mapping replica to it's state information
+     * @param writeReplicas list of write replicas {@link ReplicaInfo} instances breakdown by token range
+     * @param readReplicas  list of read replica {@link ReplicaInfo} instances breakdown by token range
+     */
+    public TokenRangeReplicasResponse(@JsonProperty("replicaState") Map<String, String> replicaState,
+                                      @JsonProperty("writeReplicas") List<ReplicaInfo> writeReplicas,
+                                      @JsonProperty("readReplicas") List<ReplicaInfo> readReplicas)
+    {
+        this.replicaState = replicaState;
+        this.writeReplicas = writeReplicas;
+        this.readReplicas = readReplicas;
+    }
+
+    /**
+     * @return returns replica to it's state information mapping
+     */
+    @JsonProperty("replicaState")
+    public Map<String, String> replicaState()
+    {
+        return replicaState;
+    }
+
+    /**
+     * @return returns the {@link ReplicaInfo} instances representing write replicas for each token range

Review Comment:
   ```suggestion
        * @return the {@link ReplicaInfo} instances representing write replicas for each token range
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1307686750


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)

Review Comment:
   Will update. This method was grouping by DC is a previous iteration which I did not rename when modified. 



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME);
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> rangeToEndpointMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        Map<List<String>, List<String>> pendingRangeMappings = storage.getPendingRangeToEndpointWithPortMap(keyspace);
+
+        Set<String> replicaSet = Stream.concat(rangeToEndpointMappings.values().stream().flatMap(List::stream),
+                                               pendingRangeMappings.values().stream().flatMap(List::stream))
+                                       .collect(Collectors.toSet());
+
+        Map<String, String> hostToDatacenter = groupHostsByDatacenter(replicaSet);
+
+        // Retrieve map of all token ranges (pending & primary) to endpoints
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas =
+        writeReplicasFromPendingRanges(rangeToEndpointMappings,
+                                       pendingRangeMappings,
+                                       hostToDatacenter,
+                                       partitioner,
+                                       keyspace);
+
+        Map<String, String> replicaToStateMap = replicaToStateMap(replicaSet, storage);
+
+        return new TokenRangeReplicasResponse(
+        replicaToStateMap,
+        writeReplicas,
+        mappingsToUnwrappedReplicaSet(rangeToEndpointMappings, hostToDatacenter, partitioner));
+    }
+
+    private Map<String, String> replicaToStateMap(Set<String> replicaSet, StorageJmxOperations storage)
+    {
+        List<String> joiningNodes = storage.getJoiningNodesWithPort();
+        List<String> leavingNodes = storage.getLeavingNodesWithPort();
+        List<String> movingNodes = storage.getMovingNodesWithPort();
+
+        String rawGossipInfo = getRawGossipInfo();
+        GossipInfoResponse gossipInfo = GossipInfoParser.parse(rawGossipInfo);
+
+        StateWithReplacement state = new StateWithReplacement(joiningNodes, leavingNodes, movingNodes, gossipInfo);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(), state::of));
+    }
+
+    private String getRawGossipInfo()
+    {
+        return jmxClient.proxy(ClusterMembershipJmxOperations.class, FAILURE_DETECTOR_OBJ_NAME)
+                        .getAllEndpointStatesWithPort();
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    writeReplicasFromPendingRanges(Map<List<String>, List<String>> naturalReplicaMappings,
+                                   Map<List<String>, List<String>> pendingRangeMappings,
+                                   Map<String, String> hostToDatacenter,
+                                   Partitioner partitioner,
+                                   String keyspace)
+    {
+        LOGGER.debug("Pending token ranges for keyspace={}, pendingRangeMappings={}", keyspace, pendingRangeMappings);
+        // Merge natural and pending range replicas to generate candidates for write-replicas
+        List<TokenRangeReplicas> replicas = Stream.concat(
+                                                  naturalReplicaMappings.entrySet().stream(),
+                                                  pendingRangeMappings.entrySet().stream())
+                                                  .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                                  new BigInteger(entry.getKey().get(0)),
+                                                  new BigInteger(entry.getKey().get(1)),
+                                                  partitioner,
+                                                  new HashSet<>(entry.getValue())))
+                                                  .flatMap(Collection::stream)
+                                                  .collect(Collectors.toList());
+
+        // Candidate write-replica mappings (merged from natural and pending ranges) are normalized
+        // by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(replicas).stream()
+                                 .map(range -> {
+                                     Map<String, List<String>> replicasByDc =
+                                     replicasByDataCenter(hostToDatacenter, range.replicaSet());
+                                     return new TokenRangeReplicasResponse.ReplicaInfo(range.start().toString(),
+                                                                                       range.end().toString(),
+                                                                                       replicasByDc);
+                                 })
+                                 .collect(Collectors.toList());
+    }
+
+    private List<TokenRangeReplicasResponse.ReplicaInfo>
+    mappingsToUnwrappedReplicaSet(Map<List<String>, List<String>> replicasByTokenRange,
+                                  Map<String, String> hostToDatacenter,
+                                  Partitioner partitioner)
+    {
+        return replicasByTokenRange.entrySet().stream()
+                                   .map(entry -> TokenRangeReplicas.generateTokenRangeReplicas(
+                                   new BigInteger(entry.getKey().get(0)),
+                                   new BigInteger(entry.getKey().get(1)),
+                                   partitioner,
+                                   new HashSet<>(entry.getValue())))
+                                   .flatMap(Collection::stream)
+                                   .sorted()
+                                   .map(rep -> {
+                                       Map<String, List<String>> replicasByDc =
+                                       replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+                                       return new TokenRangeReplicasResponse.ReplicaInfo(rep.start().toString(),
+                                                                                         rep.end().toString(),
+                                                                                         replicasByDc);
+                                   })
+                                   .collect(Collectors.toList());
+    }
+
+    private Map<String, String> groupHostsByDatacenter(Set<String> replicaSet)
+    {
+        EndpointSnitchJmxOperations endpointSnitchInfo = jmxClient.proxy(EndpointSnitchJmxOperations.class,
+                                                                         ENDPOINT_SNITCH_INFO_OBJ_NAME);
+
+        return replicaSet.stream()
+                         .collect(Collectors.toMap(Function.identity(),
+                                                   (String host) -> getDatacenter(endpointSnitchInfo, host)));
+    }
+
+    private String getDatacenter(EndpointSnitchJmxOperations endpointSnitchInfo, String host)
+    {
+        try
+        {
+            return endpointSnitchInfo.getDatacenter(host);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @NotNull
+    private static Map<String, List<String>> replicasByDataCenter(Map<String, String> hostToDatacenter,
+                                                                  Collection<String> replicas)
+    {
+        return replicas.stream().collect(Collectors.groupingBy(hostToDatacenter::get));
+    }
+
+    /**
+     * We want to identity a joining node, to replace a dead node, differently from a newly joining node. To
+     * do this we analyze gossip info and set 'Replacing' state for node replacing a dead node.
+     * {@link StateWithReplacement} is used to set replacing state for a node.
+     *
+     * <p>We are adding this state for token range replica provider endpoint. To send out replicas for a
+     * range along with state of replicas including replacing state.
+     */
+    static class StateWithReplacement extends RingProvider.State
+    {
+        private static final String STATE_REPLACING = "Replacing";

Review Comment:
   Always in favor of using enums for grouped constants such as this case. Will update.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1309076578


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.adapters.base;
+
+/**
+ * Holder class for Node related
+ */
+public class NodeInfo

Review Comment:
   Like I said earlier, I am in favor of enums (not necessarily in this holder class) for improved readability and maintainability of related constants, which in this case, relates to node states and statuses. 
   
   This is how Cassandra manages these constants as well
   https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L423
   
   @frankgh - Do you see any reasons to not go down this route?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1308014463


##########
adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java:
##########
@@ -0,0 +1,847 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    private boolean hasOverlaps(List<TokenRangeReplicas> rangeList)
+    {
+        Collections.sort(rangeList);
+        for (int c = 0, i = 1; i < rangeList.size(); i++)
+        {
+            if (rangeList.get(c++).end().compareTo(rangeList.get(i).start()) > 0) return true;
+        }
+        return false;
+    }
+
+    private boolean checkContains(List<TokenRangeReplicas> resultList, TokenRangeReplicas expected)
+    {
+        return resultList.stream()
+                         .map(TokenRangeReplicas::toString)
+                         .anyMatch(r -> r.equals(expected.toString()));
+    }
+
+    // non-overlapping ranges
+    @Test
+    public void simpleTest()
+    {
+        List<TokenRangeReplicas> simpleList = createSimpleTokenRangeReplicaList();
+        LOGGER.info("Input:" + simpleList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(simpleList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+    }
+
+    // TODO: Validate unwrapping separately

Review Comment:
   Yep this is addressed. Will remove



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1317782746


##########
README.md:
##########
@@ -49,6 +49,19 @@ The build script supports two parameters:
 Remove any versions you may not want to test with. We recommend at least the latest (released) 4.X series and `trunk`.
 See Testing for more details on how to choose which Cassandra versions to use while testing.
 
+For multi-node in-jvm dtests, network aliases will need to be setup for each Cassandra node. The tests assume each node's 
+ip address is 127.0.0.x, where x is the node id. 
+
+For example if you populated your cluster with 3 nodes, create interfaces for 127.0.0.2 and 127.0.0.3 (the first node of course uses 127.0.0.1).
+
+### macOS network aliases
+To get up and running, create a temporary alias for every node except the first:
+
+```
+sudo ifconfig lo0 alias 127.0.0.2

Review Comment:
   Addressed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313663653


##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());
+        assertThat(writeRanges).containsExactlyElementsOf(expectedRanges);
+
+        //Sorted and Overlap check
+        validateOrderAndOverlaps(writeRanges);
+        validateOrderAndOverlaps(readRanges);
+    }
+
+    private void validateOrderAndOverlaps(List<Range<BigInteger>> ranges)
+    {
+        for (int r = 0; r < ranges.size() - 1; r++)
+        {
+            assertThat(ranges.get(r).upperEndpoint()).isLessThan(ranges.get(r + 1).upperEndpoint());
+            assertThat(ranges.get(r).intersection(ranges.get(r + 1)).isEmpty()).isTrue();
+        }
+    }
+
+    protected void validateNodeStates(TokenRangeReplicasResponse mappingResponse,
+                                      Set<String> dcReplication,
+                                      Function<Integer, String> statusFunction)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int expectedReplicas = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * dcReplication.size();
+
+        AbstractCassandraTestContext cassandraTestContext = sidecarTestContext.cassandraTestContext();
+        assertThat(mappingResponse.replicaState().size()).isEqualTo(expectedReplicas);
+        for (int i = 1; i <= cassandraTestContext.cluster().size(); i++)
+        {
+            IInstanceConfig config = cassandraTestContext.cluster().get(i).config();
+
+            if (dcReplication.contains(config.localDatacenter()))
+            {
+                String ipAndPort = config.broadcastAddress().getAddress().getHostAddress() + ":"
+                                   + config.broadcastAddress().getPort();
+
+                String expectedStatus = statusFunction.apply(i);
+                assertThat(mappingResponse.replicaState().get(ipAndPort)).isEqualTo(expectedStatus);
+            }
+        }
+    }
+
+    protected UpgradeableCluster getMultiDCCluster(int numNodes,
+                                                   int numDcs,
+                                                   BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws IOException
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier =
+        MultiDcTokenSupplier.evenlyDistributedTokens(numNodes,
+                                                     numDcs,
+                                                     1);
+
+        int totalNodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return cassandraTestContext.configureAndStartCluster(
+        builder -> {
+            builder.withInstanceInitializer(initializer);
+            builder.withTokenSupplier(mdcTokenSupplier);
+            builder.withNodeIdTopology(networkTopology(totalNodeCount,
+                                                       (nodeId) -> nodeId % 2 != 0 ?
+                                                                   dcAndRack("datacenter1", "rack1") :
+                                                                   dcAndRack("datacenter2", "rack2")));
+        });
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return generateExpectedRanges(nodeCount);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(int nodeCount)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = (annotation.numDcs() > 1) ?
+                                      MultiDcTokenSupplier.evenlyDistributedTokens(
+                                      annotation.nodesPerDc() + annotation.newNodesPerDc(),
+                                      annotation.numDcs(),
+                                      1) :
+                                      TokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc() +
+                                                                            annotation.newNodesPerDc(),
+                                                                            1);
+
+        List<Range<BigInteger>> expectedRanges = new ArrayList<>();
+        BigInteger startToken = Partitioner.Murmur3.minToken;
+        BigInteger endToken = Partitioner.Murmur3.maxToken;
+        int node = 1;
+        BigInteger prevToken = new BigInteger(tokenSupplier.tokens(node++).stream().findFirst().get());
+        Range<BigInteger> firstRange = Range.openClosed(startToken, prevToken);
+        expectedRanges.add(firstRange);
+        while (node <= nodeCount)
+        {
+            BigInteger currentToken = new BigInteger(tokenSupplier.tokens(node).stream().findFirst().get());
+            expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            prevToken = currentToken;
+            node++;
+        }
+        expectedRanges.add(Range.openClosed(prevToken, endToken));
+        return expectedRanges;
+    }
+
+    protected Set<String> instancesFromReplicaSet(List<TokenRangeReplicasResponse.ReplicaInfo> replicas)
+    {
+        return replicas.stream()
+                       .flatMap(r -> r.replicasByDatacenter().values().stream())
+                       .flatMap(Collection::stream)
+                       .collect(Collectors.toSet());
+    }
+
+    protected void validateWriteReplicaMappings(List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas,
+                                              Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMapping)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        assertThat(writeReplicas).hasSize(expectedRangeMapping.get("datacenter1").size());

Review Comment:
   This validates the number of ranges match the expected no. ranges.



##########
src/test/integration/org/apache/cassandra/sidecar/routes/BaseTokenRangeIntegrationTest.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Range;
+
+import io.vertx.core.Handler;
+import io.vertx.core.buffer.Buffer;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.IntegrationTestBase;
+import org.apache.cassandra.sidecar.adapters.base.Partitioner;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+
+import static org.apache.cassandra.distributed.shared.NetworkTopology.dcAndRack;
+import static org.apache.cassandra.distributed.shared.NetworkTopology.networkTopology;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with cassandra container.
+ */
+public class BaseTokenRangeIntegrationTest extends IntegrationTestBase
+{
+
+    protected void validateTokenRanges(TokenRangeReplicasResponse mappingsResponse,
+                                       List<Range<BigInteger>> expectedRanges)
+    {
+        List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicaSet = mappingsResponse.writeReplicas();
+        List<TokenRangeReplicasResponse.ReplicaInfo> readReplicaSet = mappingsResponse.readReplicas();
+        List<Range<BigInteger>> writeRanges = writeReplicaSet.stream()
+                                                             .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                        new BigInteger(r.end())))
+                                                             .collect(Collectors.toList());
+
+        List<Range<BigInteger>> readRanges = readReplicaSet.stream()
+                                                           .map(r -> Range.openClosed(new BigInteger(r.start()),
+                                                                                      new BigInteger(r.end())))
+                                                           .collect(Collectors.toList());
+
+
+        assertThat(writeRanges.size()).isEqualTo(writeReplicaSet.size());
+        assertThat(writeRanges).containsExactlyElementsOf(expectedRanges);
+
+        //Sorted and Overlap check
+        validateOrderAndOverlaps(writeRanges);
+        validateOrderAndOverlaps(readRanges);
+    }
+
+    private void validateOrderAndOverlaps(List<Range<BigInteger>> ranges)
+    {
+        for (int r = 0; r < ranges.size() - 1; r++)
+        {
+            assertThat(ranges.get(r).upperEndpoint()).isLessThan(ranges.get(r + 1).upperEndpoint());
+            assertThat(ranges.get(r).intersection(ranges.get(r + 1)).isEmpty()).isTrue();
+        }
+    }
+
+    protected void validateNodeStates(TokenRangeReplicasResponse mappingResponse,
+                                      Set<String> dcReplication,
+                                      Function<Integer, String> statusFunction)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int expectedReplicas = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * dcReplication.size();
+
+        AbstractCassandraTestContext cassandraTestContext = sidecarTestContext.cassandraTestContext();
+        assertThat(mappingResponse.replicaState().size()).isEqualTo(expectedReplicas);
+        for (int i = 1; i <= cassandraTestContext.cluster().size(); i++)
+        {
+            IInstanceConfig config = cassandraTestContext.cluster().get(i).config();
+
+            if (dcReplication.contains(config.localDatacenter()))
+            {
+                String ipAndPort = config.broadcastAddress().getAddress().getHostAddress() + ":"
+                                   + config.broadcastAddress().getPort();
+
+                String expectedStatus = statusFunction.apply(i);
+                assertThat(mappingResponse.replicaState().get(ipAndPort)).isEqualTo(expectedStatus);
+            }
+        }
+    }
+
+    protected UpgradeableCluster getMultiDCCluster(int numNodes,
+                                                   int numDcs,
+                                                   BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws IOException
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier =
+        MultiDcTokenSupplier.evenlyDistributedTokens(numNodes,
+                                                     numDcs,
+                                                     1);
+
+        int totalNodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return cassandraTestContext.configureAndStartCluster(
+        builder -> {
+            builder.withInstanceInitializer(initializer);
+            builder.withTokenSupplier(mdcTokenSupplier);
+            builder.withNodeIdTopology(networkTopology(totalNodeCount,
+                                                       (nodeId) -> nodeId % 2 != 0 ?
+                                                                   dcAndRack("datacenter1", "rack1") :
+                                                                   dcAndRack("datacenter2", "rack2")));
+        });
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+        return generateExpectedRanges(nodeCount);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(int nodeCount)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = (annotation.numDcs() > 1) ?
+                                      MultiDcTokenSupplier.evenlyDistributedTokens(
+                                      annotation.nodesPerDc() + annotation.newNodesPerDc(),
+                                      annotation.numDcs(),
+                                      1) :
+                                      TokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc() +
+                                                                            annotation.newNodesPerDc(),
+                                                                            1);
+
+        List<Range<BigInteger>> expectedRanges = new ArrayList<>();
+        BigInteger startToken = Partitioner.Murmur3.minToken;
+        BigInteger endToken = Partitioner.Murmur3.maxToken;
+        int node = 1;
+        BigInteger prevToken = new BigInteger(tokenSupplier.tokens(node++).stream().findFirst().get());
+        Range<BigInteger> firstRange = Range.openClosed(startToken, prevToken);
+        expectedRanges.add(firstRange);
+        while (node <= nodeCount)
+        {
+            BigInteger currentToken = new BigInteger(tokenSupplier.tokens(node).stream().findFirst().get());
+            expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            prevToken = currentToken;
+            node++;
+        }
+        expectedRanges.add(Range.openClosed(prevToken, endToken));
+        return expectedRanges;
+    }
+
+    protected Set<String> instancesFromReplicaSet(List<TokenRangeReplicasResponse.ReplicaInfo> replicas)
+    {
+        return replicas.stream()
+                       .flatMap(r -> r.replicasByDatacenter().values().stream())
+                       .flatMap(Collection::stream)
+                       .collect(Collectors.toSet());
+    }
+
+    protected void validateWriteReplicaMappings(List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas,
+                                              Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMapping)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        assertThat(writeReplicas).hasSize(expectedRangeMapping.get("datacenter1").size());

Review Comment:
   This validates the number of ranges match the expected no. ranges. Will add comment



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] frankgh commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "frankgh (via GitHub)" <gi...@apache.org>.
frankgh commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1312216151


##########
src/test/resources/logback-test.xml:
##########
@@ -20,7 +20,7 @@
 
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
     <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
-      <level>INFO</level>
+      <level>DEBUG</level>

Review Comment:
   yeah, let's keep info for now until we make logging more significant for tests 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1311694963


##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse.ReplicaInfo;
+import org.apache.cassandra.sidecar.common.utils.GossipInfoParser;
+import org.jetbrains.annotations.NotNull;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.cassandra.sidecar.adapters.base.ClusterMembershipJmxOperations.FAILURE_DETECTOR_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.EndpointSnitchJmxOperations.ENDPOINT_SNITCH_INFO_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.StorageJmxOperations.STORAGE_SERVICE_OBJ_NAME;
+import static org.apache.cassandra.sidecar.adapters.base.TokenRangeReplicas.generateTokenRangeReplicas;
+
+/**
+ * Aggregates the replica-set by token range
+ */
+public class TokenRangeReplicaProvider
+{
+    private final JmxClient jmxClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicaProvider.class);
+
+    public TokenRangeReplicaProvider(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    public TokenRangeReplicasResponse tokenRangeReplicas(String keyspace, Partitioner partitioner)
+    {
+        Objects.requireNonNull(keyspace, "keyspace must be non-null");
+
+        StorageJmxOperations storage = initializeStorageOps();
+
+        // Retrieve map of primary token ranges to endpoints that describe the ring topology
+        Map<List<String>, List<String>> naturalReplicaMappings = storage.getRangeToEndpointWithPortMap(keyspace);
+        LOGGER.debug("Natural token range mappingsfor keyspace={}, pendingRangeMappings={}",

Review Comment:
   NIT: missing space in comment
   ```suggestion
           LOGGER.debug("Natural token range mappings for keyspace={}, pendingRangeMappings={}",
   ```



##########
adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java:
##########
@@ -0,0 +1,824 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    private boolean hasOverlaps(List<TokenRangeReplicas> rangeList)
+    {
+        Collections.sort(rangeList);
+        for (int c = 0, i = 1; i < rangeList.size(); i++)
+        {
+            if (rangeList.get(c++).end().compareTo(rangeList.get(i).start()) > 0) return true;
+        }
+        return false;
+    }
+
+    private boolean checkContains(List<TokenRangeReplicas> resultList, TokenRangeReplicas expected)
+    {
+        return resultList.stream()
+                         .map(TokenRangeReplicas::toString)
+                         .anyMatch(r -> r.equals(expected.toString()));
+    }

Review Comment:
   NIT: This is a personal preference of mine, but I like to see the _tests_ first in a test class, and helper methods at the end - these are details that I generally don't need to see until I'm trying to debug a failure or if the method doesn't have a good enough name (in which case, we should rename it).



##########
adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java:
##########
@@ -0,0 +1,824 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    private boolean hasOverlaps(List<TokenRangeReplicas> rangeList)
+    {
+        Collections.sort(rangeList);
+        for (int c = 0, i = 1; i < rangeList.size(); i++)
+        {
+            if (rangeList.get(c++).end().compareTo(rangeList.get(i).start()) > 0) return true;
+        }
+        return false;
+    }
+
+    private boolean checkContains(List<TokenRangeReplicas> resultList, TokenRangeReplicas expected)
+    {
+        return resultList.stream()
+                         .map(TokenRangeReplicas::toString)
+                         .anyMatch(r -> r.equals(expected.toString()));
+    }
+
+    // non-overlapping ranges
+    @Test
+    public void simpleTest()
+    {
+        List<TokenRangeReplicas> simpleList = createSimpleTokenRangeReplicaList();
+        LOGGER.info("Input:" + simpleList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(simpleList);

Review Comment:
   Given all this test does is assert that there are no overlaps in the final list, we should at least generate an input list that has some overlaps, or is there some other thing we could validate here, like the input list and output list are equal (because there should have been no change?) I think this works/passes, and would be a better assertion than just there were no overlaps (because there were no overlaps in the input), although maybe you do both?



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.adapters.base;
+
+/**
+ * Holder class for Node related
+ */
+public class NodeInfo

Review Comment:
   The `nodetool/Ring.java` is producing _human readable_ output though, and otherwise I think Arjun's `Mode` example in C* seems more appropriate. I can imagine other usage of the enum in future work potentially, but more importantly it provides a  way to group all of the things in one place. You could, I suppose, just have a bunch of `public static final String` members on an otherwise static class just to have another container for the constant strings...
   
   We could, however, do something like this:
   ```java
   public enum NodeState
   {
       JOINING,
       LEAVING,
       MOVING,
       NORMAL,
       REPLACING;
   
       private final String displayName;
   
       NodeState()
       {
           String firstChar = String.valueOf(name().charAt(0)).toLowerCase(Locale.ROOT);
           displayName = name().toLowerCase().replaceFirst(firstChar, firstChar.toUpperCase(Locale.ROOT));
       }
   
       public String displayName() {
           return displayName;
       }
   }
   ```
   And then you can use `displayName` where you need the human-readable version.
   
   This gets rid of the hard-coding of string names, but maybe is otherwise more complicated - you could also just have a helper method somewhere that does the enum -> Title case conversion.



##########
adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java:
##########
@@ -0,0 +1,472 @@
+/*
+ * 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.adapters.base;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+
+/**
+ * Representation of a token range (exclusive start and inclusive end - (start, end]) and the
+ * corresponding mapping to replica-set hosts. Static factory ensures that ranges are always unwrapped.
+ * Note: Range comparisons are used for ordering of ranges. eg. A.compareTo(B) <= 0 implies that
+ * range A occurs before range B, not their sizes.
+ */
+public class TokenRangeReplicas implements Comparable<TokenRangeReplicas>
+{
+    private final BigInteger start;
+    private final BigInteger end;
+
+    private final Partitioner partitioner;
+
+    private final Set<String> replicaSet;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicas.class);
+
+    private TokenRangeReplicas(BigInteger start, BigInteger end, Partitioner partitioner, Set<String> replicaSet)
+    {
+        this.start = start;
+        this.end = end;
+        this.partitioner = partitioner;
+        this.replicaSet = replicaSet;
+    }
+
+    public static List<TokenRangeReplicas> generateTokenRangeReplicas(BigInteger start,
+                                                                      BigInteger end,
+                                                                      Partitioner partitioner,
+                                                                      Set<String> replicaSet)
+    {
+        if (start.compareTo(end) > 0)
+        {
+            return unwrapRange(start, end, partitioner, replicaSet);
+        }
+
+        return Collections.singletonList(new TokenRangeReplicas(start, end, partitioner, replicaSet));
+    }
+
+
+    public BigInteger start()
+    {
+        return start;
+    }
+
+    public BigInteger end()
+    {
+        return end;
+    }
+
+    public Set<String> replicaSet()
+    {
+        return replicaSet;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int compareTo(@NotNull TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        int compareStart = this.start.compareTo(other.start);
+        return (compareStart != 0) ? compareStart : this.end.compareTo(other.end);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+        {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass())
+        {
+            return false;
+        }
+
+        TokenRangeReplicas that = (TokenRangeReplicas) o;
+
+        return Objects.equals(start, that.start)
+               && Objects.equals(end, that.end)
+               && partitioner == that.partitioner;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(start, end, partitioner);
+    }
+
+    private void validateRangesForComparison(@NotNull TokenRangeReplicas other)
+    {
+        if (this.partitioner != other.partitioner)
+            throw new IllegalStateException("Token ranges being compared do not have the same partitioner");
+    }
+
+    protected boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 0);
+    }
+
+    /**
+     * For subset ranges, this is used to determine if a range is larger than the other by comparing start-end lengths
+     * If both ranges end at the min, we compare starting points to determine the result.
+     * When the left range is the only one ending at min, it is always the larger one since all subsequent ranges
+     * in the sorted range list have to be smaller.
+     * <p>
+     * This method assumes that the ranges are normalized and unwrapped, i.e.
+     * 'this' comes before 'other' AND there's no wrapping around the min token
+     *
+     * @param other the next range in the range list to compare
+     * @return true if "this" range is larger than the other
+     */
+    protected boolean isLarger(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return this.end.subtract(this.start).compareTo(other.end.subtract(other.start)) > 0;
+    }
+
+    /**
+     * Determines intersection if the next range starts before the current range ends. This method assumes that
+     * the provided ranges are sorted and unwrapped.
+     * When the current range goes all the way to the end, we determine intersection if the next range starts
+     * after the current since all subsequent ranges have to be subsets.
+     *
+     * @param other the range we are currently processing to check if "this" intersects it
+     * @return true if "this" range intersects the other
+     */
+    protected boolean intersects(TokenRangeReplicas other)
+    {
+        if (this.compareTo(other) > 0)
+            throw new IllegalStateException(
+            String.format("Token ranges - (this:%s other:%s) are not ordered", this, other));
+
+        return this.end.compareTo(other.start) > 0 && this.start.compareTo(other.end) < 0; // Start exclusive (DONE)
+    }

Review Comment:
   Rather then throwing if this is < other, how about just reversing the comparison in the return statement:
   
   ```suggestion
       protected boolean intersects(TokenRangeReplicas other)
       {
           boolean inOrder = this.compareTo(other) <= 0;
           TokenRangeReplicas first = inOrder ? this : other;
           TokenRangeReplicas last = inOrder ? other : this;
   
           return first.end.compareTo(last.start) > 0 && first.start.compareTo(last.end) < 0; // Start exclusive (DONE)
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] JeetKunDoug commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "JeetKunDoug (via GitHub)" <gi...@apache.org>.
JeetKunDoug commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1313624423


##########
src/test/integration/org/apache/cassandra/testing/TestVersionSupplier.java:
##########
@@ -35,7 +35,7 @@ public class TestVersionSupplier
     Stream<TestVersion> testVersions()
     {
         // By default, we test 2 versions that will exercise oldest and newest supported versions
-        String versions = System.getProperty("cassandra.sidecar.versions_to_test", "4.0,5.0");
+        String versions = System.getProperty("cassandra.sidecar.versions_to_test", "5.1");

Review Comment:
   Before we merge, please make this "4.0,5.1" or "4.1,5.1" so we don't lose 4.X testing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra-sidecar] arjunashok commented on a diff in pull request #58: CASSANDRASC-60 Adds endpoint to serve read/write replica-sets by toke…

Posted by "arjunashok (via GitHub)" <gi...@apache.org>.
arjunashok commented on code in PR #58:
URL: https://github.com/apache/cassandra-sidecar/pull/58#discussion_r1300511317


##########
src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java:
##########
@@ -148,34 +149,21 @@ private BeforeEachCallback beforeEach()
                 int nodesPerDc = annotation.nodesPerDc();
                 int dcCount = annotation.numDcs();
                 int newNodesPerDc = annotation.newNodesPerDc(); // if the test wants to add more nodes later
+                Preconditions.checkArgument(newNodesPerDc >= 0,
+                                            "newNodesPerDc cannot be a negative number");
+                int originalNodeCount = nodesPerDc * dcCount;
                 int finalNodeCount = dcCount * (nodesPerDc + newNodesPerDc);
                 Versions.Version requestedVersion = versions.getLatest(new Semver(version.version(),
                                                                                   Semver.SemverType.LOOSE));
                 SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(version.version());
 
                 UpgradeableCluster.Builder clusterBuilder =
-                    UpgradeableCluster.build(nodesPerDc * dcCount)
-                                      .withVersion(requestedVersion)
-                                      .withDCs(dcCount)
-                                      .withDataDirCount(annotation.numDataDirsPerInstance())
-                                      .withConfig(config -> {
-                                      if (annotation.nativeTransport())
-                                      {
-                                          config.with(Feature.NATIVE_PROTOCOL);
-                                      }
-                                      if (annotation.jmx())
-                                      {
-                                          config.with(Feature.JMX);
-                                      }
-                                      if (annotation.gossip())
-                                      {
-                                          config.with(Feature.GOSSIP);
-                                      }
-                                      if (annotation.network())
-                                      {
-                                          config.with(Feature.NETWORK);
-                                      }
-                                  });
+                UpgradeableCluster.build(originalNodeCount)
+//                                  .withDynamicPortAllocation(true) // to allow parallel test runs

Review Comment:
   Temporarily disabled parallel runs as it was not playing well with in-jvm dtest based integration tests



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org