CASSANDRASC-60: Adds endpoint to serve read/write replica-sets by token-ranges

Co-authored-by: Saranya Krishnakumar <saranya_k@apple.com>

Patch by Arjun Ashok, Saranya Krishnakumar; Reviewed by Dinesh Joshi, Doug Rohrer, Francisco Guerrero, Yifan Cai for CASSANDRASC-60
diff --git a/.circleci/config.yml b/.circleci/config.yml
index 89cd80b..f6e9040 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -78,6 +78,10 @@
   integration_cassandra_40_java8:
     docker:
      - image: circleci/openjdk:8-jdk-stretch
+    environment:
+      INTEGRATION_MAX_PARALLEL_FORKS: 1
+      INTEGRATION_MAX_HEAP_SIZE: "6g"
+    resource_class: large
     steps:
      - checkout
      # Cassandra 4.0 jar seems to be missing some dependencies, so we use 4.1 here (this is what we currently do)
@@ -94,6 +98,10 @@
   integration_cassandra_41_java8:
     docker:
      - image: circleci/openjdk:8-jdk-stretch
+    environment:
+      INTEGRATION_MAX_PARALLEL_FORKS: 1
+      INTEGRATION_MAX_HEAP_SIZE: "6g"
+    resource_class: large
     steps:
      - checkout
      - run: BRANCHES="cassandra-4.1" scripts/build-dtest-jars.sh
@@ -126,6 +134,10 @@
   integration_cassandra_40_java11:
     docker:
       - image: circleci/openjdk:11-jdk-stretch
+    environment:
+      INTEGRATION_MAX_PARALLEL_FORKS: 1
+      INTEGRATION_MAX_HEAP_SIZE: "6g"
+    resource_class: large
     steps:
       - checkout
       # Cassandra 4.0 jar seems to be missing some dependencies, so we use 4.1 here (this is what we currently do)
@@ -142,10 +154,14 @@
   integration_cassandra_50_java11:
     docker:
       - image: circleci/openjdk:11-jdk-stretch
+    environment:
+      INTEGRATION_MAX_PARALLEL_FORKS: 1
+      INTEGRATION_MAX_HEAP_SIZE: "6g"
+    resource_class: large
     steps:
       - checkout
       - run: BRANCHES="cassandra-5.0" scripts/build-dtest-jars.sh
-      - run: ./gradlew -PdtestVersion=5.0-alpha1 -Dcassandra.sidecar.versions_to_test="5.0" --info checkstyleIntegrationTest spotbugsIntegrationTest integrationTest --stacktrace
+      - run: ./gradlew -PdtestVersion=5.0-alpha2 -Dcassandra.sidecar.versions_to_test="5.0" --info checkstyleIntegrationTest spotbugsIntegrationTest integrationTest --stacktrace
 
       - store_artifacts:
           path: build/reports
@@ -157,6 +173,10 @@
   integration_cassandra_trunk_java11:
     docker:
       - image: circleci/openjdk:11-jdk-stretch
+    environment:
+      INTEGRATION_MAX_PARALLEL_FORKS: 1
+      INTEGRATION_MAX_HEAP_SIZE: "6g"
+    resource_class: large
     steps:
       - checkout
       - run: BRANCHES="trunk" scripts/build-dtest-jars.sh
diff --git a/CHANGES.txt b/CHANGES.txt
index a703c40..36ec09e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 1.0.0
 -----
+ * Adds endpoint to serve read/write replica-sets by token-ranges (CASSANDRASC-60)
  * Split unit tests and integration tests in CircleCI config (CASSANDRASC-72)
  * Allow configuring permissions for uploaded SSTables (CASSANDRASC-71)
  * Refactor Sidecar configuration (CASSANDRASC-69)
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index 2a02db0..fe03e68 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -370,7 +370,7 @@
 To import the formatting configuration run the following gradle task:
 
 ```shell
-./gradlew copyCodeStyle
+./gradlew idea
 ```
 
 This will install the style settings into the `.idea` directory located at the root of the project directory.
diff --git a/README.md b/README.md
index 82c47a8..f032e69 100644
--- a/README.md
+++ b/README.md
@@ -49,6 +49,18 @@
 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:
+
+```
+ for i in {2..20}; do sudo ifconfig lo0 alias "127.0.0.${i}"; done
+```
+
 Getting started: Running The Sidecar
 --------------------------------------
 
diff --git a/adapters/base/build.gradle b/adapters/base/build.gradle
index 1e26849..8ca3a13 100644
--- a/adapters/base/build.gradle
+++ b/adapters/base/build.gradle
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+import java.nio.file.Paths
+
 plugins {
     id 'java-library'
     id 'idea'
@@ -36,6 +38,14 @@
 
 test {
     useJUnitPlatform()
+    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1
+    reports {
+        junitXml.enabled = true
+        def destDir = Paths.get(rootProject.rootDir.absolutePath, "build", "test-results", "adapters-base").toFile()
+        println("Destination directory for adapters-base tests: ${destDir}")
+        junitXml.destination = destDir
+        html.enabled = true
+    }
 }
 
 dependencies {
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
index dfe2198..68769f2 100644
--- a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/CassandraStorageOperations.java
@@ -28,6 +28,7 @@
 import org.apache.cassandra.sidecar.common.JmxClient;
 import org.apache.cassandra.sidecar.common.StorageOperations;
 import org.apache.cassandra.sidecar.common.data.RingResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
 import org.apache.cassandra.sidecar.common.dns.DnsResolver;
 import org.apache.cassandra.sidecar.common.exceptions.NodeBootstrappingException;
 import org.apache.cassandra.sidecar.common.exceptions.SnapshotAlreadyExistsException;
@@ -45,6 +46,7 @@
     private static final Logger LOGGER = LoggerFactory.getLogger(CassandraStorageOperations.class);
     protected final JmxClient jmxClient;
     protected final RingProvider ringProvider;
+    protected final TokenRangeReplicaProvider tokenRangeReplicaProvider;
 
     /**
      * Creates a new instance with the provided {@link JmxClient}
@@ -60,6 +62,7 @@
     {
         this.jmxClient = jmxClient;
         this.ringProvider = ringProvider;
+        this.tokenRangeReplicaProvider = new TokenRangeReplicaProvider(jmxClient);
     }
 
     /**
@@ -123,4 +126,13 @@
     {
         return ringProvider.ring(keyspace);
     }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public TokenRangeReplicasResponse tokenRangeReplicas(@NotNull String keyspace, @NotNull String partitioner)
+    {
+        return tokenRangeReplicaProvider.tokenRangeReplicas(keyspace, Partitioner.fromClassName(partitioner));
+    }
 }
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java
new file mode 100644
index 0000000..8d223cd
--- /dev/null
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/NodeInfo.java
@@ -0,0 +1,72 @@
+/*
+ * 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.util.Locale;
+
+/**
+ * Holder class for valid states and statuses of a node
+ */
+public class NodeInfo
+{
+    /**
+     * Represents the known states of a node
+     */
+    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;
+        }
+    }
+
+    /**
+     * Represents the statuses a node can have
+     */
+    public enum NodeStatus
+    {
+        UP,
+        DOWN;
+
+        private final String displayName;
+        NodeStatus()
+        {
+            String firstChar = String.valueOf(name().charAt(0)).toLowerCase(Locale.ROOT);
+            displayName = name().toLowerCase().replaceFirst(firstChar, firstChar.toUpperCase(Locale.ROOT));
+        }
+        public String displayName()
+        {
+            return displayName;
+        }
+    }
+}
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/Partitioner.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/Partitioner.java
new file mode 100644
index 0000000..2f4687b
--- /dev/null
+++ b/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));
+
+    public final BigInteger minToken;
+    public final BigInteger maxToken;
+
+    Partitioner(BigInteger minToken, BigInteger maxToken)
+    {
+        this.minToken = minToken;
+        this.maxToken = maxToken;
+    }
+
+    public static Partitioner fromClassName(String className)
+    {
+        switch (className)
+        {
+            case "org.apache.cassandra.dht.Murmur3Partitioner":
+                return Murmur3;
+            case "org.apache.cassandra.dht.RandomPartitioner":
+                return Random;
+            default:
+                throw new UnsupportedOperationException("Unexpected partitioner: " + className);
+        }
+    }
+}
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
index 16554d6..5d44711 100644
--- a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RingProvider.java
@@ -29,6 +29,8 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeState;
+import org.apache.cassandra.sidecar.adapters.base.NodeInfo.NodeStatus;
 import org.apache.cassandra.sidecar.common.JmxClient;
 import org.apache.cassandra.sidecar.common.data.RingEntry;
 import org.apache.cassandra.sidecar.common.data.RingResponse;
@@ -46,12 +48,6 @@
     private static final Logger LOGGER = LoggerFactory.getLogger(RingProvider.class);
     private static final String UNKNOWN_SHORT = "?";
     private static final String UNKNOWN = "Unknown";
-    private static final String STATUS_UP = "Up";
-    private static final String STATUS_DOWN = "Down";
-    private static final String STATE_JOINING = "Joining";
-    private static final String STATE_LEAVING = "Leaving";
-    private static final String STATE_MOVING = "Moving";
-    private static final String STATE_NORMAL = "Normal";
     private static final String DECIMAL_FORMAT = "##0.00%";
 
     protected final JmxClient jmxClient;
@@ -171,9 +167,9 @@
         String of(String endpoint)
         {
             if (liveNodes.contains(endpoint))
-                return STATUS_UP;
+                return NodeStatus.UP.displayName();
             if (deadNodes.contains(endpoint))
-                return STATUS_DOWN;
+                return NodeStatus.DOWN.displayName();
             return UNKNOWN_SHORT;
         }
     }
@@ -183,7 +179,7 @@
      */
     static class State
     {
-        private final Set<String> joiningNodes;
+        protected final Set<String> joiningNodes;
         private final Set<String> leavingNodes;
         private final Set<String> movingNodes;
 
@@ -197,12 +193,12 @@
         String of(String endpoint)
         {
             if (joiningNodes.contains(endpoint))
-                return STATE_JOINING;
+                return NodeState.JOINING.displayName();
             else if (leavingNodes.contains(endpoint))
-                return STATE_LEAVING;
+                return NodeState.LEAVING.displayName();
             else if (movingNodes.contains(endpoint))
-                return STATE_MOVING;
-            return STATE_NORMAL;
+                return NodeState.MOVING.displayName();
+            return NodeState.NORMAL.displayName();
         }
     }
 
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/StorageJmxOperations.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/StorageJmxOperations.java
index 369630b..d39ba8f 100644
--- a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/StorageJmxOperations.java
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/StorageJmxOperations.java
@@ -116,4 +116,22 @@
      * @param keyspaceNames an optional list of keyspaces
      */
     void clearSnapshot(String tag, String... keyspaceNames);
+
+    /**
+     * Retrieve the list of node endpoints by token range for the given keyspace
+     *
+     * @param keyspace the keyspace in Cassandra
+     * @return Returns a mapping of token range (represented by the first two entries in the key) to
+     * a list of endpoints
+     */
+    Map<List<String>, List<String>> getRangeToEndpointWithPortMap(String keyspace);
+
+    /**
+     * Retrieve the list of pending node endpoints by token range for the given keyspace
+     *
+     * @param keyspace the keyspace in Cassandra
+     * @return Returns a mapping of token range (represented by the first two entries in the key) to
+     * a list of endpoints
+     */
+    Map<List<String>, List<String>> getPendingRangeToEndpointWithPortMap(String keyspace);
 }
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java
new file mode 100644
index 0000000..f8e0d39
--- /dev/null
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProvider.java
@@ -0,0 +1,267 @@
+/*
+ * 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 interface KeyspaceToRangeMappingFunc extends Function<String, Map<List<String>, List<String>>>
+    {
+    }
+
+    protected 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();
+
+        List<TokenRangeReplicas> naturalTokenRangeReplicas =
+            getTokenRangeReplicas("Natural", keyspace, partitioner, storage::getRangeToEndpointWithPortMap);
+        // Pending ranges include bootstrap tokens and leaving endpoints as represented in the Cassandra TokenMetadata
+        List<TokenRangeReplicas> pendingTokenRangeReplicas =
+            getTokenRangeReplicas("Pending", keyspace, partitioner, storage::getPendingRangeToEndpointWithPortMap);
+
+        // 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 List<TokenRangeReplicas> getTokenRangeReplicas(String rangeType, String keyspace, Partitioner partitioner,
+                                                           KeyspaceToRangeMappingFunc rangeMappingSupplier)
+    {
+        Map<List<String>, List<String>> rangeMappings = rangeMappingSupplier.apply(keyspace);
+        LOGGER.debug(rangeType + " token range mappings for keyspace={}, rangeMappings={}", keyspace, rangeMappings);
+        return transformRangeMappings(rangeMappings, partitioner);
+    }
+
+    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 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)
+    {
+        // Candidate write-replica mappings are normalized by consolidating overlapping ranges
+        return TokenRangeReplicas.normalize(tokenRangeReplicaSet).stream()
+                                 .map(range -> buildReplicaInfo(hostToDatacenter, range))
+                                 .collect(toList());
+    }
+
+    private List<ReplicaInfo> readReplicasFromReplicaMapping(List<TokenRangeReplicas> naturalTokenRangeReplicas,
+                                                             Map<String, String> hostToDatacenter)
+    {
+        return naturalTokenRangeReplicas.stream()
+                                        .sorted()
+                                        .map(rep -> buildReplicaInfo(hostToDatacenter, rep))
+                                        .collect(toList());
+    }
+
+    @NotNull
+    private static ReplicaInfo buildReplicaInfo(Map<String, String> hostToDatacenter, TokenRangeReplicas rep)
+    {
+        Map<String, List<String>> replicasByDc = replicasByDataCenter(hostToDatacenter, rep.replicaSet());
+
+        return new ReplicaInfo(rep.start().toString(),
+                               rep.end().toString(),
+                               replicasByDc);
+    }
+
+    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)
+    {
+
+        Map<String, List<String>> dcReplicaMapping = new HashMap<>();
+
+        replicas.stream()
+            .filter(hostToDatacenter::containsKey)
+            .forEach(item ->
+                     dcReplicaMapping.computeIfAbsent(hostToDatacenter.get(item), v -> new ArrayList<>())
+                                     .add(item));
+        return dcReplicaMapping;
+    }
+
+    /**
+     * 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 final GossipInfoResponse gossipInfo;
+
+        StateWithReplacement(List<String> joiningNodes, List<String> leavingNodes, List<String> movingNodes,
+                             GossipInfoResponse gossipInfo)
+        {
+            super(joiningNodes, leavingNodes, movingNodes);
+            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.debug("Found gossipInfoEntry={}", gossipInfoEntry);
+                    String hostStatus = gossipInfoEntry.status();
+                    if (hostStatus != null && hostStatus.startsWith("BOOT_REPLACE,"))
+                    {
+                        return NodeState.REPLACING.displayName();
+                    }
+                }
+            }
+            return super.of(endpoint);
+        }
+    }
+}
diff --git a/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java
new file mode 100644
index 0000000..43fe5d8
--- /dev/null
+++ b/adapters/base/src/main/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicas.java
@@ -0,0 +1,456 @@
+/*
+ * 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);
+        }
+
+        LOGGER.info("Generating replica-map for range: {} - {} : Replicaset: {}", start, end, 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
+               && replicaSet.equals(that.replicaSet);
+    }
+
+    /**
+     * {@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");
+    }
+
+    boolean contains(TokenRangeReplicas other)
+    {
+        validateRangesForComparison(other);
+        return (other.start.compareTo(this.start) >= 0 && other.end.compareTo(this.end) <= 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
+     */
+    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;
+    }
+
+    /**
+     * 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
+     */
+    static TokenRangeReplicas processIntersectingRanges(List<TokenRangeReplicas> output,
+                                                        Iterator<TokenRangeReplicas> iter,
+                                                        TokenRangeReplicas current,
+                                                        TokenRangeReplicas next)
+    {
+        // min-heap with a comparator comparing the ends of ranges
+        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.debug("Non-empty heap while resolving intersecting ranges:" + 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
+    private static Set<String> getBatchReplicas(PriorityQueue<TokenRangeReplicas> rangeHeap)
+    {
+        return rangeHeap.stream()
+                        .map(TokenRangeReplicas::replicaSet)
+                        .flatMap(Collection::stream)
+                        .collect(Collectors.toSet());
+    }
+
+    private static Set<String> mergeReplicas(TokenRangeReplicas current, TokenRangeReplicas next)
+    {
+        Set<String> merged = new HashSet<>(current.replicaSet);
+        merged.addAll(next.replicaSet);
+        return merged;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String toString()
+    {
+        return String.format("Range(%s, %s]: %s:%s", start.toString(), end.toString(), replicaSet, partitioner);
+    }
+}
diff --git a/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java b/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java
new file mode 100644
index 0000000..ab074f3
--- /dev/null
+++ b/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicaProviderTest.java
@@ -0,0 +1,400 @@
+/*
+ * 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.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.startsWith;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for TokenRangeReplicaProvider
+ */
+public class TokenRangeReplicaProviderTest
+{
+    private static final String TEST_DC1 = "test_dc_1";
+    private static final String TEST_DC2 = "test_dc_2";
+    private static final List<String> TOKEN_RANGE1 = Arrays.asList("0", "100");
+    private static final List<String> TOKEN_RANGE2 = Arrays.asList("200", "500");
+    private static final List<String> TEST_ENDPOINTS1 = Arrays.asList("127.0.0.1:7000",
+                                                                      "127.0.0.2:7000",
+                                                                      "127.0.0.3:7000");
+    private static final List<String> TEST_ENDPOINTS2 = Arrays.asList("128.0.0.1:7000", "128.0.0.2:7000");
+    private static final List<String> TEST_MULTI_DC_ENDPOINTS = Arrays.asList("128.0.0.1:7000",
+                                                                              "128.0.0.2:7000",
+                                                                              "127.0.0.4:7000");
+    public static final String TEST_KEYSPACE = "test_keyspace";
+
+    StorageJmxOperations storageOperations;
+    EndpointSnitchJmxOperations endpointOperations;
+    ClusterMembershipJmxOperations clusterMembershipOperations;
+    JmxClient jmxClient;
+    TokenRangeReplicaProvider instance;
+
+    @BeforeEach
+    void setup()
+    {
+        storageOperations = mock(StorageJmxOperations.class);
+        endpointOperations = mock(EndpointSnitchJmxOperations.class);
+        clusterMembershipOperations = mock(ClusterMembershipJmxOperations.class);
+        jmxClient = mock(JmxClient.class);
+        instance = new TokenRangeReplicaProvider(jmxClient);
+
+        when(jmxClient.proxy(StorageJmxOperations.class, "org.apache.cassandra.db:type=StorageService"))
+        .thenReturn(storageOperations);
+        when(jmxClient.proxy(EndpointSnitchJmxOperations.class, "org.apache.cassandra.db:type=EndpointSnitchInfo"))
+        .thenReturn(endpointOperations);
+        when(jmxClient.proxy(ClusterMembershipJmxOperations.class, "org.apache.cassandra.net:type=FailureDetector"))
+        .thenReturn(clusterMembershipOperations);
+    }
+
+    @Test
+    public void replicasByTokenRangeTest() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> readReplicaMappings = new HashMap<>();
+        readReplicaMappings.put(TOKEN_RANGE1, TEST_ENDPOINTS1);
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+        Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+        when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort()).thenReturn("");
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Random);
+        assertThat(result).isNotNull();
+        assertThat(result.writeReplicas().size()).isEqualTo(1);
+        // Single token range
+        assertThat(result.readReplicas().size()).isEqualTo(1);
+        // Single DC
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().size()).isEqualTo(1);
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().get(TEST_DC1)).containsAll(TEST_ENDPOINTS1);
+        assertThat(result.replicaState().size()).isEqualTo(3);
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Normal");
+    }
+
+    @Test
+    public void replicasByTokenRangeTestMultipleDCs() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> readReplicaMappings = new HashMap<>();
+        readReplicaMappings.put(TOKEN_RANGE1, TEST_ENDPOINTS1);
+        readReplicaMappings.put(TOKEN_RANGE2, TEST_ENDPOINTS2);
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+        Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+        when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("128.0.0.1:7000", "127.0.0.2:7000"));
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort()).thenReturn(generateSampleGossip("NORMAL",
+                                                                                                         "LEAVING",
+                                                                                                         "NORMAL",
+                                                                                                         "NORMAL",
+                                                                                                         "LEAVING",
+                                                                                                         "NORMAL"));
+
+
+        when(endpointOperations.getDatacenter(startsWith("127"))).thenReturn(TEST_DC1);
+        when(endpointOperations.getDatacenter(startsWith("128"))).thenReturn(TEST_DC2);
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas("test_keyspace", Partitioner.Random);
+        assertThat(result).isNotNull();
+        assertThat(result.writeReplicas().size()).isEqualTo(2);
+        // Single token range
+        assertThat(result.readReplicas().size()).isEqualTo(2);
+        // Single DC per token range
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().size()).isEqualTo(1);
+        assertThat(result.readReplicas().get(1).replicasByDatacenter().size()).isEqualTo(1);
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().get(TEST_DC1)).containsAll(TEST_ENDPOINTS1);
+        assertThat(result.readReplicas().get(1).replicasByDatacenter().get(TEST_DC2)).containsAll(TEST_ENDPOINTS2);
+        assertThat(result.replicaState().size()).isEqualTo(5);
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("128.0.0.1:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("128.0.0.2:7000")).isEqualTo("Normal");
+    }
+
+    @Test
+    public void replicasByTokenRangeTestMultipleDCsPerTokenRange() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> readReplicaMappings = new HashMap<>();
+        readReplicaMappings.put(TOKEN_RANGE1, TEST_ENDPOINTS1);
+        readReplicaMappings.put(TOKEN_RANGE2, TEST_MULTI_DC_ENDPOINTS);
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+        Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+        when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.1:7000",
+                                                                                   "127.0.0.2:7000",
+                                                                                   "128.0.0.1:7000"));
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort())
+        .thenReturn(generateSampleGossip("LEAVING",
+                                         "LEAVING",
+                                         "NORMAL",
+                                         "NORMAL",
+                                         "LEAVING",
+                                         "NORMAL"));
+
+        when(endpointOperations.getDatacenter(startsWith("127"))).thenReturn(TEST_DC1);
+        when(endpointOperations.getDatacenter(startsWith("128"))).thenReturn(TEST_DC2);
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas("test_keyspace", Partitioner.Random);
+        assertThat(result).isNotNull();
+        assertThat(result.writeReplicas().size()).isEqualTo(2);
+        // 2 token ranges
+        assertThat(result.readReplicas().size()).isEqualTo(2);
+
+        // Validate token range has entries from both DCs
+        TokenRangeReplicasResponse.ReplicaInfo replicaInfoWithMultipleDCs =
+        result.readReplicas().stream().filter(r -> TOKEN_RANGE2.get(0).equals(r.start())).findAny().get();
+        assertThat(replicaInfoWithMultipleDCs.replicasByDatacenter().size()).isEqualTo(2);
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().get(TEST_DC1)).containsAll(TEST_ENDPOINTS1);
+        assertThat(result.readReplicas().get(1).replicasByDatacenter().get(TEST_DC2)).containsAll(TEST_ENDPOINTS2);
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.4:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("128.0.0.1:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("128.0.0.2:7000")).isEqualTo("Normal");
+    }
+
+    @Test
+    public void readAndWriteReplicasByTokenRangeTest() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> readReplicaMappings = new HashMap<>();
+        readReplicaMappings.put(TOKEN_RANGE1, TEST_ENDPOINTS1);
+
+        Map<List<String>, List<String>> writeReplicaMappings = new HashMap<>();
+        writeReplicaMappings.put(TOKEN_RANGE2, TEST_ENDPOINTS2);
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(readReplicaMappings);
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(writeReplicaMappings);
+        when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.3:7000", "128.0.0.1:7000"));
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort())
+        .thenReturn(generateSampleGossip("NORMAL",
+                                         "NORMAL",
+                                         "LEAVING",
+                                         "NORMAL",
+                                         "LEAVING",
+                                         "NORMAL"));
+
+        when(endpointOperations.getDatacenter(startsWith("127"))).thenReturn(TEST_DC1);
+        when(endpointOperations.getDatacenter(startsWith("128"))).thenReturn(TEST_DC2);
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas("test_keyspace", Partitioner.Random);
+        assertThat(result).isNotNull();
+        assertThat(result.writeReplicas().size()).isEqualTo(2);
+        assertThat(result.readReplicas().size()).isEqualTo(1);
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().size()).isEqualTo(1);
+        assertThat(result.readReplicas().get(0).replicasByDatacenter().get(TEST_DC1)).containsAll(TEST_ENDPOINTS1);
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("128.0.0.1:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("128.0.0.2:7000")).isEqualTo("Normal");
+    }
+
+    @Test
+    public void tokenRangeBeforeNodeJoins() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> rangeToEndpointWithPortMap = ImmutableMap.of(
+        Arrays.asList("3074457345618258602", "-9223372036854775808"),
+        Arrays.asList("127.0.0.1:7000", "127.0.0.2:7000", "127.0.0.3:7000"),
+        Arrays.asList("-3074457345618258603", "3074457345618258602"),
+        Arrays.asList("127.0.0.3:7000", "127.0.0.1:7000", "127.0.0.2:7000"),
+        Arrays.asList("-9223372036854775808", "-3074457345618258603"),
+        Arrays.asList("127.0.0.2:7000", "127.0.0.3:7000", "127.0.0.1:7000")
+        );
+        Map<List<String>, List<String>> pendingRangeToEndpointWithPortMap = Collections.emptyMap();
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+        .thenReturn(pendingRangeToEndpointWithPortMap);
+        when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
+        when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.4:7000", "128.0.0.2:7000"));
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort())
+        .thenReturn(generateSampleGossip("NORMAL",
+                                         "NORMAL",
+                                         "NORMAL",
+                                         "LEAVING",
+                                         "NORMAL",
+                                         "LEAVING"));
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Murmur3);
+        assertThat(result).isNotNull();
+        assertThat(result.readReplicas()).hasSize(3);
+        assertThat(validateRangeExists(result.readReplicas(), "3074457345618258602",
+                                       Long.toString(Long.MAX_VALUE))).isTrue();
+        assertThat(validateRangeExists(result.writeReplicas(), "3074457345618258602",
+                                       Long.toString(Long.MAX_VALUE))).isTrue();
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Normal");
+    }
+
+    @Test
+    public void tokenRangeDuringNodeJoin() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> rangeToEndpointWithPortMap = ImmutableMap.of(
+        Arrays.asList("3074457345618258602", "-9223372036854775808"),
+        Arrays.asList("127.0.0.1:7000", "127.0.0.2:7000", "127.0.0.3:7000"),
+        Arrays.asList("-3074457345618258603", "3074457345618258602"),
+        Arrays.asList("127.0.0.3:7000", "127.0.0.1:7000", "127.0.0.2:7000"),
+        Arrays.asList("-9223372036854775808", "-3074457345618258603"),
+        Arrays.asList("127.0.0.2:7000", "127.0.0.3:7000", "127.0.0.1:7000")
+        );
+        Map<List<String>, List<String>> pendingRangeToEndpointWithPortMap = ImmutableMap.of(
+        Arrays.asList("6148914691236517204", "9223372036854775807"),
+//        Arrays.asList("3074457345618258602", "6148914691236517204"),
+        Collections.singletonList("127.0.0.4:7000"),
+        Arrays.asList("-3074457345618258603", "3074457345618258602"),
+        Collections.singletonList("127.0.0.4:7000"),
+        Arrays.asList("-9223372036854775808", "-3074457345618258603"),
+        Collections.singletonList("127.0.0.4:7000")
+        );
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+        .thenReturn(pendingRangeToEndpointWithPortMap);
+        when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
+        when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.2:7000", "127.0.0.4:7000"));
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort()).thenReturn(generateSampleGossip("NORMAL",
+                                                                                                         "LEAVING",
+                                                                                                         "NORMAL",
+                                                                                                         "LEAVING",
+                                                                                                         "NORMAL",
+                                                                                                         "NORMAL"));
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Murmur3);
+        assertThat(result).isNotNull();
+        assertThat(result.readReplicas()).hasSize(3);
+        assertThat(result.writeReplicas()).hasSize(4);
+        // Write replicas includes the new range ending at "maxToken"
+        assertThat(validateRangeExists(result.writeReplicas(), "6148914691236517204",
+                                       Long.toString(Long.MAX_VALUE))).isTrue();
+        // Read replicas does NOT include the new range from pending ranges
+        assertThat(validateRangeExists(result.readReplicas(), "6148914691236517204",
+                                       Long.toString(Long.MAX_VALUE))).isFalse();
+        // Existing read replicas wrap-around range ends at "maxToken"
+        assertThat(validateRangeExists(result.readReplicas(), "3074457345618258602",
+                                       Long.toString(Long.MAX_VALUE))).isTrue();
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.4:7000")).isEqualTo("Leaving");
+    }
+
+    @Test
+    public void tokenRangeAfterNodeJoins() throws UnknownHostException
+    {
+        Map<List<String>, List<String>> rangeToEndpointWithPortMap = ImmutableMap.of(
+        Arrays.asList("6148914691236517204", "-9223372036854775808"),
+        Arrays.asList("127.0.0.1:7000", "127.0.0.2:7000", "127.0.0.3:7000"),
+        Arrays.asList("3074457345618258602", "6148914691236517204"),
+        Arrays.asList("127.0.0.4:7000", "127.0.0.1:7000", "127.0.0.2:7000"),
+        Arrays.asList("-3074457345618258603", "3074457345618258602"),
+        Arrays.asList("127.0.0.3:7000", "127.0.0.4:7000", "127.0.0.1:7000"),
+        Arrays.asList("-9223372036854775808", "-3074457345618258603"),
+        Arrays.asList("127.0.0.2:7000", "127.0.0.3:7000", "127.0.0.4:7000")
+        );
+        Map<List<String>, List<String>> pendingRangeToEndpointWithPortMap = Collections.emptyMap();
+
+        when(storageOperations.getRangeToEndpointWithPortMap(TEST_KEYSPACE)).thenReturn(rangeToEndpointWithPortMap);
+        when(storageOperations.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE))
+        .thenReturn(pendingRangeToEndpointWithPortMap);
+        when(endpointOperations.getDatacenter(anyString())).thenReturn(TEST_DC1);
+        when(storageOperations.getLeavingNodesWithPort()).thenReturn(Arrays.asList("127.0.0.1:7000", "128.0.0.1:7000"));
+        when(storageOperations.getJoiningNodesWithPort()).thenReturn(Collections.singletonList("127.0.0.4:7000"));
+        when(clusterMembershipOperations.getAllEndpointStatesWithPort()).thenReturn(generateSampleGossip("LEAVING",
+                                                                                                         "NORMAL",
+                                                                                                         "NORMAL",
+                                                                                                         "BOOT_REPLACE",
+                                                                                                         "NORMAL",
+                                                                                                         "NORMAL"));
+
+        TokenRangeReplicasResponse result = instance.tokenRangeReplicas(TEST_KEYSPACE, Partitioner.Murmur3);
+        assertThat(result).isNotNull();
+        assertThat(result.readReplicas()).hasSize(4);
+        assertThat(result.writeReplicas()).hasSize(4);
+        assertThat(validateRangeExists(result.readReplicas(), "6148914691236517204",
+                                       Long.toString(Long.MAX_VALUE))).isTrue();
+        assertThat(validateRangeExists(result.writeReplicas(), "6148914691236517204",
+                                       Long.toString(Long.MAX_VALUE))).isTrue();
+        assertThat(result.replicaState().get("127.0.0.1:7000")).isEqualTo("Leaving");
+        assertThat(result.replicaState().get("127.0.0.2:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.3:7000")).isEqualTo("Normal");
+        assertThat(result.replicaState().get("127.0.0.4:7000")).isEqualTo("Replacing");
+    }
+
+    private boolean validateRangeExists(List<TokenRangeReplicasResponse.ReplicaInfo> ranges, String start, String end)
+    {
+        return ranges.stream().anyMatch(r -> (r.start().equals(start) && r.end().equals(end)));
+    }
+
+    private String generateSampleGossip(String dc1Node1Status,
+                                        String dc1Node2Status,
+                                        String dc1Node3Status,
+                                        String dc1Node4Status,
+                                        String dc2Node1Status,
+                                        String dc2Node2Status)
+    {
+        return String.format("/127.0.0.1:7000%n" +
+                             "  STATUS:16:%s,9223372036854775805%n" +
+                             "  HOST_ID:21:00000000-0000-4000-8000-000000000003%n" +
+                             "  STATUS_WITH_PORT:17:%s,9223372036854775805%n" +
+                             "/127.0.0.2:7000%n" +
+                             "  STATUS:9:%s,3074457345618258601%n" +
+                             "  HOST_ID:14:00000000-0000-4000-8000-000000000002%n" +
+                             "  STATUS_WITH_PORT:10:%s,3074457345618258601%n" +
+                             "/127.0.0.3:7000%n" +
+                             "  STATUS:2:%s,-3074457345618258603%n" +
+                             "  HOST_ID:7:00000000-0000-4000-8000-000000000001%n" +
+                             "  STATUS_WITH_PORT:3:%s,-3074457345618258603%n" +
+                             "/127.0.0.4:7000%n" +
+                             "  STATUS:2:%s,-3074457345618258603%n" +
+                             "  HOST_ID:7:00000000-0000-4000-8000-000000000004%n" +
+                             "  STATUS_WITH_PORT:3:%s,-3074457345618258603%n" +
+                             "/128.0.0.1:7000%n" +
+                             "  STATUS:2:%s,-3074457345618258603%n" +
+                             "  HOST_ID:7:00000000-0000-4000-8000-000000000001%n" +
+                             "  STATUS_WITH_PORT:3:%s,-3074457345618258603%n" +
+                             "/128.0.0.2:7000%n" +
+                             "  STATUS:2:%s,-3074457345618258603%n" +
+                             "  HOST_ID:7:00000000-0000-4000-8000-000000000002%n" +
+                             "  STATUS_WITH_PORT:3:%s,-3074457345618258603%n",
+                             dc1Node1Status, dc1Node1Status, dc1Node2Status, dc1Node2Status,
+                             dc1Node3Status, dc1Node3Status, dc1Node4Status, dc1Node4Status,
+                             dc2Node1Status, dc2Node1Status, dc2Node2Status, dc1Node2Status);
+    }
+}
diff --git a/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java b/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java
new file mode 100644
index 0000000..2ebcbeb
--- /dev/null
+++ b/adapters/base/src/test/java/org/apache/cassandra/sidecar/adapters/base/TokenRangeReplicasTest.java
@@ -0,0 +1,855 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Tests for TokenRangeReplicas
+ */
+public class TokenRangeReplicasTest
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenRangeReplicasTest.class);
+
+    // 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();
+        assertThat(simpleList).containsExactlyInAnyOrderElementsOf(rangeList);
+    }
+
+    @Test
+    public void subRangeTest()
+    {
+        List<TokenRangeReplicas> subRangeList = createOverlappingTokenRangeReplicaList();
+        LOGGER.info("Input:" + subRangeList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(subRangeList);
+        LOGGER.info("Result:" + rangeList);
+
+        assertThat(hasOverlaps(rangeList)).isFalse();
+        // Validate that we have 1 additional list as a result of the splits
+        assertThat(rangeList).hasSize(subRangeList.size() + 1);
+
+        // Validate that there is a merged range with 20-30 with hosts h4-h7
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h4", "h5", "h6", "h7")));
+        // Validate absence of larger list
+        List<TokenRangeReplicas> expectedNotExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h4", "h5")));
+
+        assertThat(isPartOfRanges(expectedExists.get(0), rangeList)).isTrue();
+        assertThat(isPartOfRanges(expectedNotExists.get(0), rangeList)).isFalse();
+    }
+
+    @Test
+    public void processIntersectionTest()
+    {
+        List<TokenRangeReplicas> subRangeList = createIntersectingTokenRangeReplicaList();
+        List<TokenRangeReplicas> output = new ArrayList<>();
+        Iterator<TokenRangeReplicas> iter = subRangeList.iterator();
+        TokenRangeReplicas curr = iter.next();
+        TokenRangeReplicas next = iter.next();
+        assertThat(hasOverlaps(subRangeList)).isTrue();
+        LOGGER.info("Input:" + subRangeList);
+        TokenRangeReplicas.processIntersectingRanges(output,
+                                                     iter,
+                                                     curr,
+                                                     next);
+        LOGGER.info("Result:" + output);
+        assertThat(hasOverlaps(output)).isFalse();
+        assertThat(output).hasSize(6);
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3")));
+
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3", "h4", "h5")));
+
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3", "h4", "h5", "h6", "h7")));
+
+        assertThat(isPartOfRanges(expectedExists.get(0), output)).isTrue();
+        assertThat(isPartOfRanges(expectedExists2.get(0), output)).isTrue();
+        assertThat(isPartOfRanges(expectedExists3.get(0), output)).isTrue();
+    }
+
+    @Test
+    public void processIntersectionWithSubsetRangeTest()
+    {
+        List<TokenRangeReplicas> subRangeList = createIntersectingTokenRangeReplicaList2();
+        List<TokenRangeReplicas> output = new ArrayList<>();
+        Iterator<TokenRangeReplicas> iter = subRangeList.iterator();
+        TokenRangeReplicas curr = iter.next();
+        TokenRangeReplicas next = iter.next();
+        assertThat(hasOverlaps(subRangeList)).isTrue();
+        LOGGER.info("Input:" + subRangeList);
+        TokenRangeReplicas.processIntersectingRanges(output,
+                                                     iter,
+                                                     curr,
+                                                     next);
+        LOGGER.info("Result:" + output);
+        assertThat(hasOverlaps(output)).isFalse();
+        assertThat(output).hasSize(5);
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3")));
+
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3", "h4", "h5")));
+
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3", "h6", "h7")));
+
+        assertThat(isPartOfRanges(expectedExists.get(0), output)).isTrue();
+        assertThat(isPartOfRanges(expectedExists2.get(0), output)).isTrue();
+        assertThat(isPartOfRanges(expectedExists3.get(0), output)).isTrue();
+    }
+
+    @Test
+    public void processIntersectionWithMultipleSubsetRangeTest()
+    {
+        List<TokenRangeReplicas> subRangeList = createIntersectingTokenRangeReplicaList3();
+        List<TokenRangeReplicas> output = new ArrayList<>();
+        Iterator<TokenRangeReplicas> iter = subRangeList.iterator();
+        TokenRangeReplicas curr = iter.next();
+        TokenRangeReplicas next = iter.next();
+        assertThat(hasOverlaps(subRangeList)).isTrue();
+        LOGGER.info("Input:" + subRangeList);
+        TokenRangeReplicas.processIntersectingRanges(output,
+                                                     iter,
+                                                     curr,
+                                                     next);
+        LOGGER.info("Result:" + output);
+        assertThat(hasOverlaps(output)).isFalse();
+        assertThat(output).hasSize(6);
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("15"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3")));
+
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3", "h4", "h5")));
+
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("35"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h9", "h1", "h2", "h3", "h4", "h5")));
+
+        assertThat(isPartOfRanges(expectedExists.get(0), output)).isTrue();
+        assertThat(isPartOfRanges(expectedExists2.get(0), output)).isTrue();
+        assertThat(isPartOfRanges(expectedExists3.get(0), output)).isTrue();
+    }
+
+    // Validate merge-split resulting from 2 ranges overlapping
+    @Test
+    public void partialOverlapTest()
+    {
+        List<TokenRangeReplicas> partialOverlapList = createPartialOverlappingTokenRangeReplicaList();
+        LOGGER.info("Input:" + partialOverlapList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(partialOverlapList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+        // Validate that we have 1 additional list as a result of the splits
+        assertThat(rangeList).hasSize(partialOverlapList.size() + 1);
+
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("15"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5")));
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h4", "h5", "h6", "h7")));
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7")));
+        List<TokenRangeReplicas> expectedNotExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5")));
+        List<TokenRangeReplicas> expectedNotExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7")));
+
+        assertThat(checkContains(rangeList, expectedExists.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists2.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists3.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedNotExists.get(0))).isFalse();
+        assertThat(checkContains(rangeList, expectedNotExists2.get(0))).isFalse();
+    }
+
+    // Validate merge-split resulting from 3 consecutive ranges overlapping
+    @Test
+    public void multiOverlapTest()
+    {
+        List<TokenRangeReplicas> multiOverlapList = createMultipleOverlappingTokenRangeReplicaList();
+        LOGGER.info("Input:" + multiOverlapList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(multiOverlapList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+        // Validate that we have 1 additional list as a result of the splits
+        assertThat(rangeList).hasSize(multiOverlapList.size() + 1);
+
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("15"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h1", "h2", "h3", "h4", "h5")));
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("25"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h4", "h5", "h6", "h7")));
+        List<TokenRangeReplicas> expectedNotExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("25"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5")));
+
+        assertThat(checkContains(rangeList, expectedExists.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists2.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedNotExists.get(0))).isFalse();
+    }
+
+    //     Validate merge-split from wrapped overlapping ranges
+    @Test
+    public void wrappedMultiOverlapTest()
+    {
+        List<TokenRangeReplicas> overlapList = createUnwrappedMultipleOverlappingTokenRangeReplicaList();
+        LOGGER.info("Input:" + overlapList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(overlapList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+
+        // Validate that we have 1 additional ranges as a result of the merges and splits
+        assertThat(rangeList).hasSize(overlapList.size() + 1);
+
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h9", "h1", "h2", "h3")));
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("35"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9", "h6", "h7")));
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("35"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9")));
+        List<TokenRangeReplicas> expectedNotExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9")));
+
+        assertThat(checkContains(rangeList, expectedExists.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists2.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists3.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedNotExists.get(0))).isFalse();
+    }
+
+    @Test
+    public void wrappedOverlapTest()
+    {
+        List<TokenRangeReplicas> wrappedOverlapList = createWrappedOverlappingTokenRangeReplicaList();
+        LOGGER.info("Input:" + wrappedOverlapList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(wrappedOverlapList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+
+        // Split & Merge should result in same number of ranges
+        // (-1, 10] and (-1, 20] results in (-1, 10] and (10, 20]
+        // (35, max] and (40, mx] results in (35, 40] and (40, max]
+        assertThat(rangeList).hasSize(wrappedOverlapList.size());
+
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h9", "h1", "h2", "h3")));
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("35"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9")));
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9", "h4", "h5")));
+        List<TokenRangeReplicas> expectedNotExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5")));
+
+        assertThat(checkContains(rangeList, expectedExists.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists2.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists3.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedNotExists.get(0))).isFalse();
+    }
+
+    // Validate case when the partitioner min token does not match the least token value in the ring
+    @Test
+    public void wrappedOverlapNonMatchingMinTokenTest()
+    {
+        List<TokenRangeReplicas> wrappedOverlapList = createWrappedOvlNonMatchingMinTokenList();
+        LOGGER.info("Input:" + wrappedOverlapList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(wrappedOverlapList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+
+        // Split & Merge should result in same number of ranges
+        // (-1, 5] and (-1, 10] results in (-1, 5] and (5, 10]
+        // (5, 20] results in (10, 20]
+        // (35, max] and (40, mx] results in (35, 40] and (40, max]
+        assertThat(rangeList).hasSize(wrappedOverlapList.size());
+
+        List<TokenRangeReplicas> expectedExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h9", "h1", "h2", "h3")));
+        // New Token range resulting from non-matching min token
+        List<TokenRangeReplicas> expectedExistsNew =
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("5"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(
+                                                      Arrays.asList("h9", "h1", "h2", "h3")));
+        List<TokenRangeReplicas> expectedExists2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("35"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Collections.singletonList("h9")));
+        // Other split resulting from new range
+        List<TokenRangeReplicas> expectedExists3 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("5"),
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9", "h4", "h5")));
+        List<TokenRangeReplicas> expectedNotExists =
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5")));
+
+        assertThat(checkContains(rangeList, expectedExists.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExistsNew.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists2.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists3.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedNotExists.get(0))).isFalse();
+    }
+
+    // Test using actual ranges from a 3 node cluster
+    @Test
+    public void wrappedActualOverlapTest()
+    {
+        List<TokenRangeReplicas> createdList = new ArrayList<>();
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Murmur3.minToken,
+                                                      new BigInteger("-3074457345618258603"),
+                                                      Partitioner.Murmur3,
+                                                      new HashSet<>(Arrays.asList("h2", "h3", "h1"))));
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("-3074457345618258603"),
+                                                      new BigInteger("3074457345618258602"),
+                                                      Partitioner.Murmur3,
+                                                      new HashSet<>(Arrays.asList("h3", "h1", "h2"))));
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("3074457345618258602"),
+                                                      new BigInteger("6148914691236517204"),
+                                                      Partitioner.Murmur3,
+                                                      new HashSet<>(Arrays.asList("h9"))));
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("-6148914691236517204"),
+                                                      new BigInteger("3074457345618258602"),
+                                                      Partitioner.Murmur3,
+                                                      new HashSet<>(Arrays.asList("h9"))));
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Murmur3.minToken,
+                                                      new BigInteger("-3074457345618258603"),
+                                                      Partitioner.Murmur3,
+                                                      new HashSet<>(Arrays.asList("h10"))));
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("6148914691236517204"),
+                                                      Partitioner.Murmur3.minToken,
+                                                      Partitioner.Murmur3,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))));
+
+        LOGGER.info("Input:" + createdList);
+        List<TokenRangeReplicas> rangeList = TokenRangeReplicas.normalize(createdList);
+        LOGGER.info("Result:" + rangeList);
+        assertThat(hasOverlaps(rangeList)).isFalse();
+        assertThat(rangeList).hasSize(5);
+
+        List<TokenRangeReplicas> expectedExists = TokenRangeReplicas.generateTokenRangeReplicas(
+        Partitioner.Murmur3.minToken, new BigInteger("-6148914691236517204"),
+        Partitioner.Murmur3, new HashSet<>(Arrays.asList("h10", "h1", "h2", "h3")));
+        List<TokenRangeReplicas> expectedExists2 = TokenRangeReplicas.generateTokenRangeReplicas(
+        new BigInteger("-3074457345618258603"), new BigInteger("3074457345618258602"),
+        Partitioner.Murmur3, new HashSet<>(Arrays.asList("h9", "h1", "h2", "h3")));
+        List<TokenRangeReplicas> expectedExists3 = TokenRangeReplicas.generateTokenRangeReplicas(
+        new BigInteger("3074457345618258602"), new BigInteger("6148914691236517204"),
+        Partitioner.Murmur3, new HashSet<>(Arrays.asList("h9")));
+        List<TokenRangeReplicas> expectedExists4 = TokenRangeReplicas.generateTokenRangeReplicas(
+        new BigInteger("6148914691236517204"), Partitioner.Murmur3.maxToken,
+        Partitioner.Murmur3, new HashSet<>(Arrays.asList("h1", "h2", "h3")));
+        List<TokenRangeReplicas> expectedNotExists = TokenRangeReplicas.generateTokenRangeReplicas(
+        new BigInteger("3074457345618258602"), Partitioner.Murmur3.minToken,
+        Partitioner.Murmur3, new HashSet<>(Arrays.asList("h1", "h2", "h3")));
+
+        assertThat(checkContains(rangeList, expectedExists.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists2.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists3.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedExists4.get(0))).isTrue();
+        assertThat(checkContains(rangeList, expectedNotExists.get(0))).isFalse();
+    }
+
+    @Test
+    void testSubsetRelationship()
+    {
+        // wraps around
+        List<TokenRangeReplicas> range1 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(100), BigInteger.valueOf(-100), Partitioner.Murmur3, new HashSet<>());
+        List<TokenRangeReplicas> range2 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(100), Partitioner.Murmur3.maxToken, Partitioner.Murmur3, new HashSet<>());
+        List<TokenRangeReplicas> range3 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(100), BigInteger.valueOf(150), Partitioner.Murmur3, new HashSet<>());
+        List<TokenRangeReplicas> range4 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(120), BigInteger.valueOf(150), Partitioner.Murmur3, new HashSet<>());
+        assertThat(range1).hasSize(2);
+        assertThat(range2).hasSize(1);
+        assertThat(range3).hasSize(1);
+        assertThat(range4).hasSize(1);
+        assertThat(isPartOfRanges(range2.get(0), range1)).isTrue();
+        assertThat(isPartOfRanges(range3.get(0), range2)).isTrue();
+        assertThat(isPartOfRanges(range2.get(0), range3)).isFalse();
+        assertThat(isPartOfRanges(range4.get(0), range3)).isTrue();
+        assertThat(isPartOfRanges(range3.get(0), range4)).isFalse();
+    }
+
+    @Test
+    void testIntersectRanges()
+    {
+        // Simple Intersection
+        List<TokenRangeReplicas> range1 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(1), BigInteger.valueOf(10), Partitioner.Murmur3, new HashSet<>());
+        List<TokenRangeReplicas> range2 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(9), BigInteger.valueOf(12), Partitioner.Murmur3, new HashSet<>());
+        assertThat(intersectsWithRanges(range2.get(0), range1)).isTrue();
+        // intersect check with out-of-order ranges
+        assertThat(intersectsWithRanges(range1.get(0), range2)).isTrue();
+
+        // Intersection at border
+        List<TokenRangeReplicas> range3 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(1), BigInteger.valueOf(10), Partitioner.Murmur3, new HashSet<>());
+        List<TokenRangeReplicas> range4 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(11), BigInteger.valueOf(20), Partitioner.Murmur3, new HashSet<>());
+
+
+        assertThat(intersectsWithRanges(range4.get(0), range3)).isFalse();
+        // intersect check with out-of-order ranges
+        assertThat(intersectsWithRanges(range3.get(0), range4)).isFalse();
+
+        // Intersection as superset
+        List<TokenRangeReplicas> range5 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(1000), Partitioner.Murmur3.maxToken, Partitioner.Murmur3, new HashSet<>());
+        List<TokenRangeReplicas> range6 = TokenRangeReplicas.generateTokenRangeReplicas(
+        BigInteger.valueOf(200000), BigInteger.valueOf(300000), Partitioner.Murmur3, new HashSet<>());
+
+        assertThat(intersectsWithRanges(range6.get(0), range5)).isTrue();
+    }
+
+    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()));
+    }
+
+    private boolean isPartOfRanges(TokenRangeReplicas range, List<TokenRangeReplicas> rangeList)
+    {
+        return rangeList.stream().anyMatch(r -> r.contains(range));
+    }
+
+    private boolean intersectsWithRanges(TokenRangeReplicas range, List<TokenRangeReplicas> rangeList)
+    {
+        return rangeList.stream().anyMatch(r -> r.intersects(range));
+    }
+
+    private List<TokenRangeReplicas> createSimpleTokenRangeReplicaList()
+    {
+        List<TokenRangeReplicas> simpleList = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      Partitioner.Random.minToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0)
+        );
+        assertThat(hasOverlaps(simpleList)).isFalse();
+        return simpleList;
+    }
+
+    // 2. Simple single overlap (consuming) => superset + no changes to others [Merge]
+    private List<TokenRangeReplicas> createOverlappingTokenRangeReplicaList()
+    {
+        List<TokenRangeReplicas> rangeWithOverlaps = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(Partitioner.Random.minToken,
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9"))).get(0)
+        );
+        assertThat(hasOverlaps(rangeWithOverlaps)).isTrue();
+        return rangeWithOverlaps;
+    }
+
+    // 3. Single overlap - cutting [Merge + Split]
+    private List<TokenRangeReplicas> createPartialOverlappingTokenRangeReplicaList()
+    {
+        List<TokenRangeReplicas> rangeWithOverlaps = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("-1"),
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("20"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9"))).get(0)
+        );
+        assertThat(hasOverlaps(rangeWithOverlaps)).isTrue();
+        return rangeWithOverlaps;
+    }
+
+    // 4. Multi-overlaps
+    private List<TokenRangeReplicas> createMultipleOverlappingTokenRangeReplicaList()
+    {
+        List<TokenRangeReplicas> rangeWithOverlaps = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("-1"),
+                                                      new BigInteger("15"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("25"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9"))).get(0)
+        );
+        assertThat(hasOverlaps(rangeWithOverlaps)).isTrue();
+        return rangeWithOverlaps;
+        
+    }
+
+    // 5. Overlaps w/ wrap-around
+    private List<TokenRangeReplicas> createUnwrappedMultipleOverlappingTokenRangeReplicaList()
+    {
+
+        List<TokenRangeReplicas> createdList = new ArrayList<>();
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("-1"),
+                                                      new BigInteger("15"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))));
+
+        createdList.addAll(TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                                         new BigInteger("20"),
+                                                                         Partitioner.Random,
+                                                                         new HashSet<>(Arrays.asList("h4", "h5"))));
+
+        createdList.addAll(TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                                         new BigInteger("35"),
+                                                                         Partitioner.Random,
+                                                                         new HashSet<>(Arrays.asList("h6", "h7"))));
+
+        List<TokenRangeReplicas> wrappedList =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9")));
+        assertThat(wrappedList).hasSize(2);
+
+
+        createdList.addAll(wrappedList);
+        assertThat(hasWrappedRange(createdList)).isFalse();
+        assertThat(hasOverlaps(createdList)).isTrue();
+        return createdList;
+    }
+
+    private boolean hasWrappedRange(List<TokenRangeReplicas> createdList)
+    {
+        return createdList.stream().anyMatch(r -> r.start().compareTo(r.end()) > 0);
+    }
+
+    private List<TokenRangeReplicas> createIntersectingTokenRangeReplicaList()
+    {
+        List<TokenRangeReplicas> rangeWithOverlaps = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("60"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("80"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9"))).get(0)
+        );
+        assertThat(hasOverlaps(rangeWithOverlaps)).isTrue();
+        return rangeWithOverlaps;
+    }
+
+    private List<TokenRangeReplicas> createIntersectingTokenRangeReplicaList2()
+    {
+        List<TokenRangeReplicas> rangeWithOverlaps = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      new BigInteger("80"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9"))).get(0)
+        );
+        assertThat(hasOverlaps(rangeWithOverlaps)).isTrue();
+        return rangeWithOverlaps;
+
+    }
+
+    private List<TokenRangeReplicas> createIntersectingTokenRangeReplicaList3()
+    {
+        List<TokenRangeReplicas> rangeWithOverlaps = Arrays.asList(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("10"),
+                                                      new BigInteger("40"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("15"),
+                                                      new BigInteger("35"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h4", "h5"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                      new BigInteger("30"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h6", "h7"))).get(0),
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("30"),
+                                                      Partitioner.Random.maxToken,
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9"))).get(0)
+        );
+        assertThat(hasOverlaps(rangeWithOverlaps)).isTrue();
+        return rangeWithOverlaps;
+
+    }
+
+
+    private List<TokenRangeReplicas> createWrappedOverlappingTokenRangeReplicaList()
+    {
+        List<TokenRangeReplicas> createdList = new ArrayList<>();
+
+        createdList.addAll(TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("-1"),
+                                                                         new BigInteger("20"),
+                                                                         Partitioner.Random,
+                                                                         new HashSet<>(Arrays.asList("h4", "h5"))));
+        createdList.addAll(TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                                         new BigInteger("35"),
+                                                                         Partitioner.Random,
+                                                                         new HashSet<>(Arrays.asList("h6", "h7"))));
+        createdList.addAll(
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      new BigInteger("-1"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3"))));
+
+
+        List<TokenRangeReplicas> wrappedRange =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("35"),
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9")));
+        createdList.addAll(wrappedRange);
+        assertThat(wrappedRange).hasSize(2);
+        // We should not have wrapped ranges as generateTokenRangeReplicas unwraps them, and we validate this below
+        assertThat(hasWrappedRange(createdList)).isFalse();
+        assertThat(hasOverlaps(createdList)).isTrue();
+        return createdList;
+    }
+
+    private List<TokenRangeReplicas> createWrappedOvlNonMatchingMinTokenList()
+    {
+        List<TokenRangeReplicas> createdList = new ArrayList<>();
+
+        createdList.addAll(TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("5"),
+                                                                         new BigInteger("20"),
+                                                                         Partitioner.Random,
+                                                                         new HashSet<>(Arrays.asList("h4", "h5"))));
+        createdList.addAll(TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("20"),
+                                                                         new BigInteger("35"),
+                                                                         Partitioner.Random,
+                                                                         new HashSet<>(Arrays.asList("h6", "h7"))));
+
+        List<TokenRangeReplicas> wrappedRange1 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("40"),
+                                                      new BigInteger("5"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h1", "h2", "h3")));
+
+        List<TokenRangeReplicas> wrappedRange2 =
+        TokenRangeReplicas.generateTokenRangeReplicas(new BigInteger("35"),
+                                                      new BigInteger("10"),
+                                                      Partitioner.Random,
+                                                      new HashSet<>(Arrays.asList("h9")));
+        createdList.addAll(wrappedRange1);
+        assertThat(wrappedRange1).hasSize(2);
+        createdList.addAll(wrappedRange2);
+        assertThat(wrappedRange2).hasSize(2);
+        // We should not have wrapped ranges as generateTokenRangeReplicas unwraps them, and we validate this below
+        assertThat(hasWrappedRange(createdList)).isFalse();
+        assertThat(hasOverlaps(createdList)).isTrue();
+
+        return createdList;
+    }
+}
diff --git a/build.gradle b/build.gradle
index 9f55443..f0ca960 100644
--- a/build.gradle
+++ b/build.gradle
@@ -54,6 +54,12 @@
 
 println("Using DTest jar: ${dtestVersion}")
 
+def integrationMaxHeapSize = System.getenv("INTEGRATION_MAX_HEAP_SIZE") ?: "8g"
+println("Using ${integrationMaxHeapSize} maxHeapSize")
+
+def integrationMaxParallelForks = (System.getenv("INTEGRATION_MAX_PARALLEL_FORKS") ?: "4") as int
+println("Using ${integrationMaxParallelForks} maxParallelForks")
+
 // Force checkstyle, rat, and spotBugs to run before test tasks for faster feedback
 def codeCheckTasks = task("codeCheckTasks")
 
@@ -342,7 +348,8 @@
     classpath = sourceSets.integrationTest.runtimeClasspath
     shouldRunAfter test
     forkEvery = 1 // DTest framework tends to have issues without forkEvery test class
-    maxHeapSize = "8g"
+    maxHeapSize = integrationMaxHeapSize
+    maxParallelForks = integrationMaxParallelForks
 }
 
 // copy the user documentation to the final build
diff --git a/client/build.gradle b/client/build.gradle
index 6e59530..d7564cd 100644
--- a/client/build.gradle
+++ b/client/build.gradle
@@ -42,6 +42,7 @@
     testLogging {
         events "passed", "skipped", "failed"
     }
+    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1
     reports {
         junitXml.enabled = true
         def destDir = Paths.get(rootProject.rootDir.absolutePath, "build", "test-results", "client").toFile()
diff --git a/client/src/main/java/org/apache/cassandra/sidecar/client/RequestContext.java b/client/src/main/java/org/apache/cassandra/sidecar/client/RequestContext.java
index 8d09d17..f595fb1 100644
--- a/client/src/main/java/org/apache/cassandra/sidecar/client/RequestContext.java
+++ b/client/src/main/java/org/apache/cassandra/sidecar/client/RequestContext.java
@@ -36,6 +36,7 @@
 import org.apache.cassandra.sidecar.client.request.SchemaRequest;
 import org.apache.cassandra.sidecar.client.request.SidecarHealthRequest;
 import org.apache.cassandra.sidecar.client.request.TimeSkewRequest;
+import org.apache.cassandra.sidecar.client.request.TokenRangeReplicasRequest;
 import org.apache.cassandra.sidecar.client.request.UploadSSTableRequest;
 import org.apache.cassandra.sidecar.client.retry.ExponentialBackoffRetryPolicy;
 import org.apache.cassandra.sidecar.client.retry.NoRetryPolicy;
@@ -234,6 +235,18 @@
         }
 
         /**
+         * Sets the {@code request} to be a {@link TokenRangeReplicasRequest} and returns a reference to this Builder
+         * enabling method chaining.
+         *
+         * @param keyspace the keyspace in Cassandra
+         * @return a reference to this Builder
+         */
+        public Builder tokenRangeReplicasRequest(String keyspace)
+        {
+            return request(new TokenRangeReplicasRequest(keyspace));
+        }
+
+        /**
          * Sets the {@code request} to be a {@link NodeSettingsRequest} and returns a reference to this Builder
          * enabling method chaining.
          *
diff --git a/client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java b/client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java
index ec3f3cc..d9d77f8 100644
--- a/client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java
+++ b/client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java
@@ -41,6 +41,7 @@
 import org.apache.cassandra.sidecar.common.data.SSTableImportResponse;
 import org.apache.cassandra.sidecar.common.data.SchemaResponse;
 import org.apache.cassandra.sidecar.common.data.TimeSkewResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
 import org.apache.cassandra.sidecar.common.utils.HttpRange;
 
 /**
@@ -191,6 +192,24 @@
     }
 
     /**
+     * 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());
+    }
+
+    /**
      * Executes the list snapshot files request including secondary index files using the default retry policy and
      * provided {@code instance}
      *
diff --git a/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java b/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java
new file mode 100644
index 0000000..712ac75
--- /dev/null
+++ b/client/src/main/java/org/apache/cassandra/sidecar/client/request/TokenRangeReplicasRequest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.client.request;
+
+import io.netty.handler.codec.http.HttpMethod;
+import org.apache.cassandra.sidecar.common.ApiEndpointsV1;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+
+/**
+ * Represents a request to retrieve information from the token-range replicas endpoint
+ */
+public class TokenRangeReplicasRequest extends DecodableRequest<TokenRangeReplicasResponse>
+{
+    /**
+     * Constructs a new request to retrieve information by keyspace from token-range replicas endpoint
+     */
+    public TokenRangeReplicasRequest(String keyspace)
+    {
+        super(ApiEndpointsV1.KEYSPACE_TOKEN_MAPPING_ROUTE.replaceAll(ApiEndpointsV1.KEYSPACE_PATH_PARAM, keyspace));
+    }
+
+    @Override
+    public HttpMethod method()
+    {
+        return HttpMethod.GET;
+    }
+}
diff --git a/client/src/testFixtures/java/org/apache/cassandra/sidecar/client/SidecarClientTest.java b/client/src/testFixtures/java/org/apache/cassandra/sidecar/client/SidecarClientTest.java
index 68eb1b3..d8a7f4e 100644
--- a/client/src/testFixtures/java/org/apache/cassandra/sidecar/client/SidecarClientTest.java
+++ b/client/src/testFixtures/java/org/apache/cassandra/sidecar/client/SidecarClientTest.java
@@ -65,6 +65,7 @@
 import org.apache.cassandra.sidecar.common.data.SSTableImportResponse;
 import org.apache.cassandra.sidecar.common.data.SchemaResponse;
 import org.apache.cassandra.sidecar.common.data.TimeSkewResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
 import org.apache.cassandra.sidecar.common.utils.HttpRange;
 
 import static io.netty.handler.codec.http.HttpResponseStatus.ACCEPTED;
@@ -324,6 +325,45 @@
     }
 
     @Test
+    public void testTokenRangeReplicasFromReplicaSet() throws Exception
+    {
+        String keyspace = "test";
+        String nodeWithPort = "127.0.0.1:7000";
+        String expectedRangeStart = "-9223372036854775808";
+        String expectedRangeEnd = "9223372036854775807";
+        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);
+        TokenRangeReplicasResponse.ReplicaInfo writeReplica = result.writeReplicas().get(0);
+        assertThat(writeReplica.start()).isEqualTo(expectedRangeStart);
+        assertThat(writeReplica.end()).isEqualTo(expectedRangeEnd);
+        assertThat(writeReplica.replicasByDatacenter()).containsKey("datacenter1");
+        assertThat(writeReplica.replicasByDatacenter().get("datacenter1")).containsExactly(nodeWithPort);
+        assertThat(result.readReplicas()).hasSize(1);
+        TokenRangeReplicasResponse.ReplicaInfo readReplica = result.readReplicas().get(0);
+        assertThat(readReplica.start()).isEqualTo(expectedRangeStart);
+        assertThat(readReplica.end()).isEqualTo(expectedRangeEnd);
+        assertThat(readReplica.replicasByDatacenter()).containsKey("datacenter1");
+        assertThat(readReplica.replicasByDatacenter().get("datacenter1")).containsExactly(nodeWithPort);
+        assertThat(result.replicaState()).hasSize(1);
+        assertThat(result.replicaState().get(nodeWithPort)).isEqualTo("NORMAL");
+
+        validateResponseServed(ApiEndpointsV1.KEYSPACE_TOKEN_MAPPING_ROUTE.replaceAll(
+        ApiEndpointsV1.KEYSPACE_PATH_PARAM, keyspace));
+    }
+
+    @Test
     public void testListSnapshotFiles() throws Exception
     {
         String responseAsString = "{\"snapshotFilesInfo\":[{\"size\":15,\"host\":\"localhost1\",\"port\":2020," +
diff --git a/common/build.gradle b/common/build.gradle
index cd8e5f0..e9e8d08 100644
--- a/common/build.gradle
+++ b/common/build.gradle
@@ -41,6 +41,7 @@
 
 test {
     useJUnitPlatform()
+    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1
     reports {
         junitXml.enabled = true
         def destDir = Paths.get(rootProject.rootDir.absolutePath, "build", "test-results", "common").toFile()
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV1.java b/common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV1.java
index f68c469..77ad601 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV1.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV1.java
@@ -78,6 +78,8 @@
     public static final String GOSSIP_INFO_ROUTE = API_V1 + CASSANDRA + "/gossip";
     public static final String TIME_SKEW_ROUTE = API_V1 + "/time-skew";
 
+    public static final String KEYSPACE_TOKEN_MAPPING_ROUTE = API_V1 + PER_KEYSPACE + "/token-range-replicas";
+
     private ApiEndpointsV1()
     {
         throw new IllegalStateException(getClass() + " is a constants container and shall not be instantiated");
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSessionProvider.java b/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSessionProvider.java
index 7b9b9db..586651d 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSessionProvider.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSessionProvider.java
@@ -126,13 +126,21 @@
         return localSession;
     }
 
-    public synchronized void close()
+    public Session close()
     {
+        Session localSession;
+        synchronized (this)
+        {
+            localSession = this.localSession;
+            this.localSession = null;
+        }
+
         if (localSession != null)
         {
             try
             {
                 localSession.getCluster().closeAsync().get(1, TimeUnit.MINUTES);
+                localSession.closeAsync().get(1, TimeUnit.MINUTES);
             }
             catch (InterruptedException e)
             {
@@ -146,8 +154,8 @@
             {
                 throw propagateCause(e);
             }
-            localSession = null;
         }
+        return localSession;
     }
 
     static RuntimeException propagateCause(ExecutionException e)
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java b/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java
index 9f7feaa..360389f 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java
@@ -47,6 +47,8 @@
  */
 public class CassandraAdapterDelegate implements ICassandraAdapter, Host.StateListener
 {
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraAdapterDelegate.class);
+
     private final String sidecarVersion;
     private final CassandraVersionProvider versionProvider;
     private final CQLSessionProvider cqlSessionProvider;
@@ -54,8 +56,6 @@
     private SimpleCassandraVersion currentVersion;
     private ICassandraAdapter adapter;
     private volatile NodeSettings nodeSettings = null;
-
-    private static final Logger logger = LoggerFactory.getLogger(CassandraAdapterDelegate.class);
     private final AtomicBoolean registered = new AtomicBoolean(false);
     private final AtomicBoolean isHealthCheckActive = new AtomicBoolean(false);
 
@@ -107,7 +107,7 @@
         }
         else
         {
-            logger.debug("Skipping health check because there's an active check at the moment");
+            LOGGER.debug("Skipping health check because there's an active check at the moment");
         }
     }
 
@@ -116,7 +116,7 @@
         Session activeSession = cqlSessionProvider.localCql();
         if (activeSession == null)
         {
-            logger.info("No local CQL session is available. Cassandra is down presumably.");
+            LOGGER.info("No local CQL session is available. Cassandra is down presumably.");
             nodeSettings = null;
             return;
         }
@@ -141,14 +141,14 @@
                 adapter = versionProvider.cassandra(releaseVersion)
                                          .create(cqlSessionProvider, jmxClient);
                 nodeSettings = newNodeSettings;
-                logger.info("Cassandra version change detected (from={} to={}). New adapter loaded={}",
+                LOGGER.info("Cassandra version change detected (from={} to={}). New adapter loaded={}",
                             previousVersion, currentVersion, adapter);
             }
-            logger.debug("Cassandra version {}", releaseVersion);
+            LOGGER.debug("Cassandra version {}", releaseVersion);
         }
         catch (IllegalArgumentException | NoHostAvailableException e)
         {
-            logger.error("Unexpected error connecting to Cassandra instance.", e);
+            LOGGER.error("Unexpected error connecting to Cassandra instance.", e);
             // The cassandra node is down.
             // Unregister the host listener and nullify the session in order to get a new object.
             nodeSettings = null;
@@ -240,21 +240,19 @@
 
     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);
         }
     }
 
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java b/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
index 2b37514..91196ed 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
@@ -26,6 +26,7 @@
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
+import java.util.concurrent.TimeUnit;
 import java.util.function.BooleanSupplier;
 import java.util.function.Supplier;
 import javax.management.JMX;
@@ -40,6 +41,11 @@
 import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
+import com.google.common.util.concurrent.Uninterruptibles;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.sidecar.common.exceptions.JmxAuthenticationException;
 import org.jetbrains.annotations.VisibleForTesting;
 
@@ -51,6 +57,7 @@
     public static final String JMX_PROTOCOL = "rmi";
     public static final String JMX_URL_PATH_FORMAT = "/jndi/rmi://%s:%d/jmxrmi";
     public static final String REGISTRY_CONTEXT_SOCKET_FACTORY = "com.sun.jndi.rmi.factory.socket";
+    private static final Logger LOGGER = LoggerFactory.getLogger(JmxClient.class);
     private final JMXServiceURL jmxServiceURL;
     private MBeanServerConnection mBeanServerConnection;
     private boolean connected = false;
@@ -58,6 +65,8 @@
     private final Supplier<String> roleSupplier;
     private final Supplier<String> passwordSupplier;
     private final BooleanSupplier enableSslSupplier;
+    private final int jmxConnectionMaxRetries;
+    private final long jmxConnectionRetryDelayMillis;
 
     /**
      * Creates a new client with the provided {@code host} and {@code port}.
@@ -84,6 +93,24 @@
         this(buildJmxServiceURL(host, port), () -> role, () -> password, () -> enableSSl);
     }
 
+    /**
+     * Creates a new client with the provided parameters
+     *
+     * @param host                       the host of the JMX service
+     * @param port                       the port of the JMX service
+     * @param role                       the JMX role used for authentication
+     * @param password                   the JMX role password used for authentication
+     * @param enableSSl                  true if SSL is enabled for JMX, false otherwise
+     * @param connectionMaxRetries       the maximum number of connection retries before failing to connect
+     * @param connectionRetryDelayMillis the number of milliseconds to delay between connection retries
+     */
+    public JmxClient(String host, int port, String role, String password,
+                     boolean enableSSl, int connectionMaxRetries, long connectionRetryDelayMillis)
+    {
+        this(buildJmxServiceURL(host, port), () -> role, () -> password, () -> enableSSl,
+             connectionMaxRetries, connectionRetryDelayMillis);
+    }
+
     @VisibleForTesting
     JmxClient(JMXServiceURL jmxServiceURL)
     {
@@ -110,10 +137,22 @@
                      Supplier<String> passwordSupplier,
                      BooleanSupplier enableSslSupplier)
     {
+        this(jmxServiceURL, roleSupplier, passwordSupplier, enableSslSupplier, 20, 1000);
+    }
+
+    public JmxClient(JMXServiceURL jmxServiceURL,
+                     Supplier<String> roleSupplier,
+                     Supplier<String> passwordSupplier,
+                     BooleanSupplier enableSslSupplier,
+                     int jmxConnectionMaxRetries,
+                     long jmxConnectionRetryDelayMillis)
+    {
         this.jmxServiceURL = Objects.requireNonNull(jmxServiceURL, "jmxServiceURL is required");
         this.roleSupplier = Objects.requireNonNull(roleSupplier, "roleSupplier is required");
         this.passwordSupplier = Objects.requireNonNull(passwordSupplier, "passwordSupplier is required");
         this.enableSslSupplier = Objects.requireNonNull(enableSslSupplier, "enableSslSupplier is required");
+        this.jmxConnectionMaxRetries = jmxConnectionMaxRetries;
+        this.jmxConnectionRetryDelayMillis = jmxConnectionRetryDelayMillis;
     }
 
     /**
@@ -158,32 +197,57 @@
 
     private void connect()
     {
-        try
+        int attempts = 1;
+        int maxAttempts = jmxConnectionMaxRetries;
+        Throwable lastThrown = null;
+        while (attempts <= maxAttempts)
         {
-            jmxConnector = JMXConnectorFactory.connect(jmxServiceURL, buildJmxEnv());
-            jmxConnector.addConnectionNotificationListener(this, null, null);
-            mBeanServerConnection = jmxConnector.getMBeanServerConnection();
-            connected = true;
+            try
+            {
+                connectInternal(attempts);
+                return;
+            }
+            // Unrecoverable errors
+            catch (SecurityException securityException)
+            {
+                // If we can't connect because we have bad credentials, don't retry
+                connected = false;
+                String errorMessage = securityException.getMessage() != null
+                                      ? securityException.getMessage()
+                                      : "JMX Authentication failed";
+                throw new JmxAuthenticationException(errorMessage, securityException);
+            }
+            catch (RuntimeException runtimeException)
+            {
+                // catch exceptions coming from the lambdas and wrap them in a JmxAuthenticationException
+                throw new JmxAuthenticationException(runtimeException);
+            }
+            // Anything else is recoverable so we should retry.
+            catch (Throwable t)
+            {
+                lastThrown = t;
+                if (attempts < maxAttempts)
+                {
+                    LOGGER.info("Could not connect to JMX on {} after {} attempts. Will retry.",
+                                jmxServiceURL, attempts, t);
+                    Uninterruptibles.sleepUninterruptibly(jmxConnectionRetryDelayMillis, TimeUnit.MILLISECONDS);
+                }
+                attempts++;
+            }
         }
-        catch (IOException iox)
-        {
-            connected = false;
-            throw new RuntimeException(String.format("Failed to connect to JMX endpoint %s", jmxServiceURL),
-                                       iox);
-        }
-        catch (SecurityException securityException)
-        {
-            connected = false;
-            String errorMessage = securityException.getMessage() != null
-                                  ? securityException.getMessage()
-                                  : "JMX Authentication failed";
-            throw new JmxAuthenticationException(errorMessage, securityException);
-        }
-        catch (RuntimeException runtimeException)
-        {
-            // catch exceptions coming from the lambdas and wrap them in a JmxAuthenticationException
-            throw new JmxAuthenticationException(runtimeException);
-        }
+        String error = "Failed to connect to JMX, which was unreachable after " + attempts + " attempts.";
+        LOGGER.error(error, lastThrown);
+        throw new RuntimeException(error, lastThrown);
+    }
+
+    private void connectInternal(int currentAttempt) throws IOException
+    {
+        jmxConnector = JMXConnectorFactory.connect(jmxServiceURL, buildJmxEnv());
+        jmxConnector.addConnectionNotificationListener(this, null, null);
+        mBeanServerConnection = jmxConnector.getMBeanServerConnection();
+        connected = true;
+        LOGGER.info("Connected to JMX server at {} after {} attempt(s)",
+                    jmxServiceURL, currentAttempt);
     }
 
     @Override
@@ -259,12 +323,17 @@
     }
 
     @Override
-    public synchronized void close() throws IOException
+    public void close() throws IOException
     {
-        JMXConnector connector = jmxConnector;
+        JMXConnector connector;
+        synchronized (this)
+        {
+            connector = jmxConnector;
+            jmxConnector = null;
+            connected = false;
+        }
         if (connector != null)
         {
-            jmxConnector = null;
             connector.close();
         }
     }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java b/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
index c1c5a84..75d9980 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
@@ -22,6 +22,7 @@
 import java.util.Map;
 
 import org.apache.cassandra.sidecar.common.data.RingResponse;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -58,4 +59,13 @@
      * @throws UnknownHostException when hostname of peer Cassandra nodes cannot be resolved
      */
     RingResponse ring(@Nullable String keyspace) throws UnknownHostException;
+
+    /**
+     * Get the token ranges and the corresponding read and write replicas by datacenter
+     * @param keyspace the keyspace in the Cassandra database
+     * @param partitioner token partitioner used for token assignment
+     * @return token range to read and write replica mappings
+     */
+    TokenRangeReplicasResponse tokenRangeReplicas(@NotNull String keyspace,
+                                                  @NotNull String partitioner);
 }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/data/GossipInfoResponse.java b/common/src/main/java/org/apache/cassandra/sidecar/common/data/GossipInfoResponse.java
index 2656219..8fb6676 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/data/GossipInfoResponse.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/data/GossipInfoResponse.java
@@ -54,6 +54,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
+     * fail. This method attempts to lookup the value by prepending the {@code slash} at the beginning.
+     * If the lookup fails, it defaults to the original behavior.
+     *
+     * @param key the key whose associated value is to be returned
+     * @return {@link GossipInfo}
+     */
+    @Override
+    public GossipInfo get(Object key)
+    {
+        if (key instanceof String)
+        {
+            String keyAsString = (String) key;
+            if (keyAsString.length() > 0 && keyAsString.charAt(0) != '/')
+            {
+                GossipInfo value = super.get("/" + key);
+                if (value != null)
+                {
+                    return value;
+                }
+            }
+        }
+        return super.get(key);
+    }
+
+    /**
      * Data accessor for reading Gossip states
      */
     public static class GossipInfo extends HashMap<String, String>
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java b/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java
new file mode 100644
index 0000000..e00f0ab
--- /dev/null
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasRequest.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.common.data;
+
+import java.util.Objects;
+
+/**
+ *  Data class to define the json format of token ranges endpoint request.
+ */
+public class TokenRangeReplicasRequest
+{
+    private final String keyspace;
+
+    /**
+     * Constructs a ring request with {@code keyspace} parameter.
+     *
+     * @param keyspace  the keyspace in Cassandra
+     */
+    public TokenRangeReplicasRequest(String keyspace)
+    {
+        this.keyspace = keyspace;
+    }
+
+    /**
+     * @return the keyspace for the request
+     */
+    public String keyspace()
+    {
+        return keyspace;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TokenRangeReplicasRequest that = (TokenRangeReplicasRequest) o;
+        return Objects.equals(keyspace, that.keyspace);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(keyspace);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String toString()
+    {
+        return "TokenRangeReplicasRequest{" +
+               "keyspace='" + keyspace +
+               "'}";
+    }
+}
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasResponse.java b/common/src/main/java/org/apache/cassandra/sidecar/common/data/TokenRangeReplicasResponse.java
new file mode 100644
index 0000000..585e729
--- /dev/null
+++ b/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 the replica to state information mapping
+     */
+    @JsonProperty("replicaState")
+    public Map<String, String> replicaState()
+    {
+        return replicaState;
+    }
+
+    /**
+     * @return the {@link ReplicaInfo} instances representing write replicas for each token range
+     */
+    @JsonProperty("writeReplicas")
+    public List<ReplicaInfo> writeReplicas()
+    {
+        return writeReplicas;
+    }
+
+    /**
+     * @return the {@link ReplicaInfo} instances representing read replicas for each token range
+     */
+    @JsonProperty("readReplicas")
+    public List<ReplicaInfo> readReplicas()
+    {
+        return readReplicas;
+    }
+
+    /**
+     * Class representing replica instances for a token range grouped by datacenter
+     */
+    public static class ReplicaInfo
+    {
+        private final String start;
+        private final String end;
+        private final Map<String, List<String>> replicasByDatacenter;
+
+        public ReplicaInfo(@JsonProperty("start") String start,
+                           @JsonProperty("end") String end,
+                           @JsonProperty("replicas") Map<String, List<String>> replicasByDc)
+        {
+            this.start = start;
+            this.end = end;
+            this.replicasByDatacenter = replicasByDc;
+        }
+
+        /**
+         * @return the start value of the token range
+         */
+        @JsonProperty("start")
+        public String start()
+        {
+            return start;
+        }
+
+        /**
+         * @return the end value of the token range
+         */
+        @JsonProperty("end")
+        public String end()
+        {
+            return end;
+        }
+
+        /**
+         * @return mapping of datacenter to a list of replicas that map to the token range
+         */
+        @JsonProperty("replicasByDatacenter")
+        public Map<String, List<String>> replicasByDatacenter()
+        {
+            return replicasByDatacenter;
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            ReplicaInfo that = (ReplicaInfo) o;
+            return start.equals(that.start)
+                   && end.equals(that.end)
+                   && replicasByDatacenter.equals(that.replicasByDatacenter);
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        public int hashCode()
+        {
+            return Objects.hash(start, end, replicasByDatacenter);
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        public String toString()
+        {
+            return "ReplicaInfo{" +
+                   "start='" + start + '\'' +
+                   ", end='" + end + '\'' +
+                   ", replicasByDatacenter=" + replicasByDatacenter +
+                   '}';
+        }
+    }
+}
diff --git a/scripts/build-dtest-jars.sh b/scripts/build-dtest-jars.sh
index 9d9118d..e8492b2 100755
--- a/scripts/build-dtest-jars.sh
+++ b/scripts/build-dtest-jars.sh
@@ -19,10 +19,10 @@
 
 set -xe
 CANDIDATE_BRANCHES=(
-  "cassandra-4.0:d13b3ef61b9afbd04878c988c7b722507674228c"
-  "cassandra-4.1:8666265521c97a5e726c9d38762028a14325e4dc"
-  "cassandra-5.0:410018ab165b54c378648d52fb4ec815c557e80e"
-  "trunk:cbaef9094e83364e6812c65b8411ff7dbffaf9c6"
+  "cassandra-4.0:1f79c8492528f01bcc5f88951a1cc9e0d7265c54"
+  "cassandra-4.1:725655dda2776fef35567496a6e331102eb7610d"
+  "cassandra-5.0:f19dd0bb1309c35535876e8f0f996ad2b76adda5"
+  "trunk:2a5e1b77c9f8a205dbec1afdea3f4ed1eaf6a4eb"
 )
 BRANCHES=( ${BRANCHES:-cassandra-4.0 cassandra-4.1 cassandra-5.0 trunk} )
 echo ${BRANCHES[*]}
diff --git a/src/main/dist/conf/sidecar.yaml b/src/main/dist/conf/sidecar.yaml
index ddd9adc..4542d77 100644
--- a/src/main/dist/conf/sidecar.yaml
+++ b/src/main/dist/conf/sidecar.yaml
@@ -91,7 +91,9 @@
       name: "sidecar-internal-worker-pool"
       size: 20
       max_execution_time_millis: 900000 # 15 minutes
-
+  jmx:
+    max_retries: 3
+    retry_delay_millis: 200
 #
 # Enable SSL configuration (Disabled by default)
 #
diff --git a/src/main/java/org/apache/cassandra/sidecar/MainModule.java b/src/main/java/org/apache/cassandra/sidecar/MainModule.java
index 09a8955..f530898 100644
--- a/src/main/java/org/apache/cassandra/sidecar/MainModule.java
+++ b/src/main/java/org/apache/cassandra/sidecar/MainModule.java
@@ -57,6 +57,7 @@
 import org.apache.cassandra.sidecar.common.utils.SidecarVersionProvider;
 import org.apache.cassandra.sidecar.config.CassandraInputValidationConfiguration;
 import org.apache.cassandra.sidecar.config.InstanceConfiguration;
+import org.apache.cassandra.sidecar.config.JmxConfiguration;
 import org.apache.cassandra.sidecar.config.ServiceConfiguration;
 import org.apache.cassandra.sidecar.config.SidecarConfiguration;
 import org.apache.cassandra.sidecar.config.SslConfiguration;
@@ -71,6 +72,7 @@
 import org.apache.cassandra.sidecar.routes.SnapshotsHandler;
 import org.apache.cassandra.sidecar.routes.StreamSSTableComponentHandler;
 import org.apache.cassandra.sidecar.routes.TimeSkewHandler;
+import org.apache.cassandra.sidecar.routes.TokenRangeReplicaMapHandler;
 import org.apache.cassandra.sidecar.routes.cassandra.NodeSettingsHandler;
 import org.apache.cassandra.sidecar.routes.sstableuploads.SSTableCleanupHandler;
 import org.apache.cassandra.sidecar.routes.sstableuploads.SSTableImportHandler;
@@ -136,6 +138,7 @@
                               SnapshotsHandler snapshotsHandler,
                               SchemaHandler schemaHandler,
                               RingHandler ringHandler,
+                              TokenRangeReplicaMapHandler tokenRangeHandler,
                               LoggerHandler loggerHandler,
                               GossipInfoHandler gossipInfoHandler,
                               TimeSkewHandler timeSkewHandler,
@@ -212,6 +215,9 @@
         router.put(ApiEndpointsV1.SSTABLE_UPLOAD_ROUTE)
               .handler(ssTableUploadHandler);
 
+        router.get(ApiEndpointsV1.KEYSPACE_TOKEN_MAPPING_ROUTE)
+              .handler(tokenRangeHandler);
+
         router.put(ApiEndpointsV1.SSTABLE_IMPORT_ROUTE)
               .handler(ssTableImportHandler);
 
@@ -264,10 +270,14 @@
         List<InstanceMetadata> instanceMetadataList =
         configuration.cassandraInstances()
                      .stream()
-                     .map(cassandraInstance -> buildInstanceMetadata(cassandraInstance,
-                                                                     cassandraVersionProvider,
-                                                                     healthCheckFrequencyMillis,
-                                                                     sidecarVersionProvider.sidecarVersion()))
+                     .map(cassandraInstance -> {
+                         JmxConfiguration jmxConfiguration = configuration.serviceConfiguration().jmxConfiguration();
+                         return buildInstanceMetadata(cassandraInstance,
+                                                      cassandraVersionProvider,
+                                                      healthCheckFrequencyMillis,
+                                                      sidecarVersionProvider.sidecarVersion(),
+                                                      jmxConfiguration);
+                     })
                      .collect(Collectors.toList());
 
         return new InstancesConfigImpl(instanceMetadataList, dnsResolver);
@@ -348,12 +358,14 @@
      * @param versionProvider            a Cassandra version provider
      * @param healthCheckFrequencyMillis the health check frequency configuration in milliseconds
      * @param sidecarVersion             the version of the Sidecar from the current binary
+     * @param jmxConfiguration
      * @return the build instance metadata object
      */
     private static InstanceMetadata buildInstanceMetadata(InstanceConfiguration cassandraInstance,
                                                           CassandraVersionProvider versionProvider,
                                                           int healthCheckFrequencyMillis,
-                                                          String sidecarVersion)
+                                                          String sidecarVersion,
+                                                          JmxConfiguration jmxConfiguration)
     {
         String host = cassandraInstance.host();
         int port = cassandraInstance.port();
@@ -363,7 +375,9 @@
                                             cassandraInstance.jmxPort(),
                                             cassandraInstance.jmxRole(),
                                             cassandraInstance.jmxRolePassword(),
-                                            cassandraInstance.jmxSslEnabled());
+                                            cassandraInstance.jmxSslEnabled(),
+                                            jmxConfiguration.maxRetries(),
+                                            jmxConfiguration.retryDelayMillis());
         CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(versionProvider,
                                                                          session,
                                                                          jmxClient,
diff --git a/src/main/java/org/apache/cassandra/sidecar/config/JmxConfiguration.java b/src/main/java/org/apache/cassandra/sidecar/config/JmxConfiguration.java
new file mode 100644
index 0000000..866fe29
--- /dev/null
+++ b/src/main/java/org/apache/cassandra/sidecar/config/JmxConfiguration.java
@@ -0,0 +1,35 @@
+/*
+ * 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.config;
+
+/**
+ * General JMX connectivity configuration that is not instance-specific.
+ */
+public interface JmxConfiguration
+{
+    /**
+     * @return the maximum number of connection retry attempts to make before failing
+     */
+    int maxRetries();
+
+    /**
+     * @return the delay, in milliseconds, between retry attempts
+     */
+    long retryDelayMillis();
+}
diff --git a/src/main/java/org/apache/cassandra/sidecar/config/ServiceConfiguration.java b/src/main/java/org/apache/cassandra/sidecar/config/ServiceConfiguration.java
index 8d7af15..d108c73 100644
--- a/src/main/java/org/apache/cassandra/sidecar/config/ServiceConfiguration.java
+++ b/src/main/java/org/apache/cassandra/sidecar/config/ServiceConfiguration.java
@@ -91,4 +91,9 @@
     {
         return workerPoolsConfiguration().get(INTERNAL_POOL);
     }
+
+    /**
+     * @return the system-wide JMX configuration settings
+     */
+    JmxConfiguration jmxConfiguration();
 }
diff --git a/src/main/java/org/apache/cassandra/sidecar/config/yaml/JmxConfigurationImpl.java b/src/main/java/org/apache/cassandra/sidecar/config/yaml/JmxConfigurationImpl.java
new file mode 100644
index 0000000..022d4ec
--- /dev/null
+++ b/src/main/java/org/apache/cassandra/sidecar/config/yaml/JmxConfigurationImpl.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.config.yaml;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.cassandra.sidecar.config.JmxConfiguration;
+
+/**
+ * General JMX connectivity configuration that is not instance-specific.
+ */
+public class JmxConfigurationImpl implements JmxConfiguration
+{
+
+    @JsonProperty("max_retries")
+    protected final int maxRetries;
+
+    @JsonProperty("retry_delay_millis")
+    protected final long retryDelayMillis;
+
+    public JmxConfigurationImpl()
+    {
+        this(3, 200L);
+    }
+
+    public JmxConfigurationImpl(int maxRetries, long retryDelayMillis)
+    {
+        this.maxRetries = maxRetries;
+        this.retryDelayMillis = retryDelayMillis;
+    }
+
+    /**
+     * @return the maximum number of connection retry attempts to make before failing
+     */
+    @JsonProperty("max_retries")
+    public int maxRetries()
+    {
+        return maxRetries;
+    }
+
+    /**
+     * @return the delay, in milliseconds, between retry attempts
+     */
+    @JsonProperty("retry_delay_millis")
+    public long retryDelayMillis()
+    {
+        return retryDelayMillis;
+    }
+}
diff --git a/src/main/java/org/apache/cassandra/sidecar/config/yaml/ServiceConfigurationImpl.java b/src/main/java/org/apache/cassandra/sidecar/config/yaml/ServiceConfigurationImpl.java
index 0716fbc..9b829e5 100644
--- a/src/main/java/org/apache/cassandra/sidecar/config/yaml/ServiceConfigurationImpl.java
+++ b/src/main/java/org/apache/cassandra/sidecar/config/yaml/ServiceConfigurationImpl.java
@@ -24,6 +24,7 @@
 import java.util.concurrent.TimeUnit;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.cassandra.sidecar.config.JmxConfiguration;
 import org.apache.cassandra.sidecar.config.SSTableImportConfiguration;
 import org.apache.cassandra.sidecar.config.SSTableUploadConfiguration;
 import org.apache.cassandra.sidecar.config.ServiceConfiguration;
@@ -87,6 +88,9 @@
     @JsonProperty(value = WORKER_POOLS_PROPERTY, required = true)
     protected final Map<String, ? extends WorkerPoolConfiguration> workerPoolsConfiguration;
 
+    @JsonProperty("jmx")
+    protected final JmxConfiguration jmxConfiguration;
+
     public ServiceConfigurationImpl()
     {
         this(DEFAULT_HOST,
@@ -97,7 +101,8 @@
              new ThrottleConfigurationImpl(),
              new SSTableUploadConfigurationImpl(),
              new SSTableImportConfigurationImpl(),
-             DEFAULT_WORKER_POOLS_CONFIGURATION);
+             DEFAULT_WORKER_POOLS_CONFIGURATION,
+             new JmxConfigurationImpl());
     }
 
     public ServiceConfigurationImpl(SSTableImportConfiguration ssTableImportConfiguration)
@@ -110,12 +115,14 @@
              new ThrottleConfigurationImpl(),
              new SSTableUploadConfigurationImpl(),
              ssTableImportConfiguration,
-             DEFAULT_WORKER_POOLS_CONFIGURATION);
+             DEFAULT_WORKER_POOLS_CONFIGURATION,
+             new JmxConfigurationImpl());
     }
 
     public ServiceConfigurationImpl(String host,
                                     ThrottleConfiguration throttleConfiguration,
-                                    SSTableUploadConfiguration ssTableUploadConfiguration)
+                                    SSTableUploadConfiguration ssTableUploadConfiguration,
+                                    JmxConfiguration jmxConfiguration)
     {
         this(host,
              DEFAULT_PORT,
@@ -125,7 +132,8 @@
              throttleConfiguration,
              ssTableUploadConfiguration,
              new SSTableImportConfigurationImpl(),
-             DEFAULT_WORKER_POOLS_CONFIGURATION);
+             DEFAULT_WORKER_POOLS_CONFIGURATION,
+             jmxConfiguration);
     }
 
     public ServiceConfigurationImpl(int requestIdleTimeoutMillis,
@@ -141,7 +149,8 @@
              new ThrottleConfigurationImpl(),
              ssTableUploadConfiguration,
              new SSTableImportConfigurationImpl(),
-             DEFAULT_WORKER_POOLS_CONFIGURATION);
+             DEFAULT_WORKER_POOLS_CONFIGURATION,
+             new JmxConfigurationImpl());
     }
 
     public ServiceConfigurationImpl(String host,
@@ -152,7 +161,8 @@
                                     ThrottleConfiguration throttleConfiguration,
                                     SSTableUploadConfiguration ssTableUploadConfiguration,
                                     SSTableImportConfiguration ssTableImportConfiguration,
-                                    Map<String, ? extends WorkerPoolConfiguration> workerPoolsConfiguration)
+                                    Map<String, ? extends WorkerPoolConfiguration> workerPoolsConfiguration,
+                                    JmxConfiguration jmxConfiguration)
     {
         this.host = host;
         this.port = port;
@@ -162,6 +172,7 @@
         this.throttleConfiguration = throttleConfiguration;
         this.ssTableUploadConfiguration = ssTableUploadConfiguration;
         this.ssTableImportConfiguration = ssTableImportConfiguration;
+        this.jmxConfiguration = jmxConfiguration;
         if (workerPoolsConfiguration == null || workerPoolsConfiguration.isEmpty())
         {
             this.workerPoolsConfiguration = DEFAULT_WORKER_POOLS_CONFIGURATION;
@@ -172,6 +183,20 @@
         }
     }
 
+    public ServiceConfigurationImpl(String host)
+    {
+        this(host,
+             DEFAULT_PORT,
+             DEFAULT_REQUEST_IDLE_TIMEOUT_MILLIS,
+             DEFAULT_REQUEST_TIMEOUT_MILLIS,
+             DEFAULT_ALLOWABLE_SKEW_IN_MINUTES,
+             new ThrottleConfigurationImpl(),
+             new SSTableUploadConfigurationImpl(),
+             new SSTableImportConfigurationImpl(),
+             DEFAULT_WORKER_POOLS_CONFIGURATION,
+             new JmxConfigurationImpl());
+    }
+
     /**
      * Sidecar's HTTP REST API listen address
      */
@@ -264,4 +289,15 @@
     {
         return workerPoolsConfiguration;
     }
+
+    /**
+     * @return the general JMX configuration options (not per-instance)
+     */
+    @Override
+    @JsonProperty("jmx")
+    public JmxConfiguration jmxConfiguration()
+    {
+        return jmxConfiguration;
+    }
+
 }
diff --git a/src/main/java/org/apache/cassandra/sidecar/config/yaml/SidecarConfigurationImpl.java b/src/main/java/org/apache/cassandra/sidecar/config/yaml/SidecarConfigurationImpl.java
index 48b2c10..8ee8c37 100644
--- a/src/main/java/org/apache/cassandra/sidecar/config/yaml/SidecarConfigurationImpl.java
+++ b/src/main/java/org/apache/cassandra/sidecar/config/yaml/SidecarConfigurationImpl.java
@@ -33,6 +33,7 @@
 import org.apache.cassandra.sidecar.config.CassandraInputValidationConfiguration;
 import org.apache.cassandra.sidecar.config.HealthCheckConfiguration;
 import org.apache.cassandra.sidecar.config.InstanceConfiguration;
+import org.apache.cassandra.sidecar.config.JmxConfiguration;
 import org.apache.cassandra.sidecar.config.KeyStoreConfiguration;
 import org.apache.cassandra.sidecar.config.SSTableImportConfiguration;
 import org.apache.cassandra.sidecar.config.SSTableUploadConfiguration;
@@ -210,7 +211,9 @@
                                     .addAbstractTypeMapping(ThrottleConfiguration.class,
                                                             ThrottleConfigurationImpl.class)
                                     .addAbstractTypeMapping(WorkerPoolConfiguration.class,
-                                                            WorkerPoolConfigurationImpl.class);
+                                                            WorkerPoolConfigurationImpl.class)
+                                    .addAbstractTypeMapping(JmxConfiguration.class,
+                                                            JmxConfigurationImpl.class);
 
         ObjectMapper mapper = new ObjectMapper(new YAMLFactory())
                               .configure(DeserializationFeature.ACCEPT_SINGLE_VALUE_AS_ARRAY, true)
diff --git a/src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java b/src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java
new file mode 100644
index 0000000..f4b824e
--- /dev/null
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/TokenRangeReplicaMapHandler.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+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 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.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.HttpExceptions;
+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 operations = delegate.storageOperations();
+        Metadata metadata = delegate.metadata();
+        if (operations == null || metadata == null)
+        {
+            context.fail(cassandraServiceUnavailable());
+            return;
+        }
+
+        executorPools.service()
+                     .executeBlocking(promise -> context.json(operations.tokenRangeReplicas(request.keyspace(),
+                                                                                            metadata.getPartitioner()))
+                     ).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(HttpExceptions.wrapHttpException(HttpResponseStatus.NOT_FOUND, cause.getMessage()));
+            return;
+        }
+
+        super.processFailure(cause, context, host, remoteAddress, request);
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java b/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
index 5ed8268..d1868c1 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestBase.java
@@ -24,6 +24,7 @@
 import java.nio.file.Paths;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -31,6 +32,7 @@
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import com.google.common.collect.ImmutableMap;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.slf4j.Logger;
@@ -49,7 +51,7 @@
 import org.apache.cassandra.sidecar.common.data.QualifiedTableName;
 import org.apache.cassandra.sidecar.common.dns.DnsResolver;
 import org.apache.cassandra.sidecar.testing.CassandraSidecarTestContext;
-import org.apache.cassandra.testing.CassandraTestContext;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -67,13 +69,15 @@
 
     protected static final String TEST_KEYSPACE = "testkeyspace";
     private static final String TEST_TABLE_PREFIX = "testtable";
+
+    protected static final int DEFAULT_RF = 3;
     private static final AtomicInteger TEST_TABLE_ID = new AtomicInteger(0);
     protected CassandraSidecarTestContext sidecarTestContext;
 
     @BeforeEach
-    void setup(CassandraTestContext cassandraTestContext) throws InterruptedException
+    void setup(AbstractCassandraTestContext cassandraTestContext) throws InterruptedException
     {
-        this.sidecarTestContext = CassandraSidecarTestContext.from(cassandraTestContext, DnsResolver.DEFAULT);
+        sidecarTestContext = CassandraSidecarTestContext.from(cassandraTestContext, DnsResolver.DEFAULT);
         Injector injector = Guice.createInjector(Modules
                                                  .override(new MainModule())
                                                  .with(new IntegrationTestModule(this.sidecarTestContext)));
@@ -83,8 +87,11 @@
 
         VertxTestContext context = new VertxTestContext();
         server.listen(server.actualPort(), "127.0.0.1", context.succeeding(p -> {
-            instancesConfig.instances()
-                           .forEach(instanceMetadata -> instanceMetadata.delegate().healthCheck());
+            if (sidecarTestContext.isClusterBuilt())
+            {
+                healthCheck(instancesConfig);
+            }
+            sidecarTestContext.registerInstanceConfigListener(instances -> healthCheck(instances));
             context.completeNow();
         }));
 
@@ -111,32 +118,38 @@
         tester.accept(client);
 
         // wait until the test completes
-        assertThat(context.awaitCompletion(30, TimeUnit.SECONDS)).isTrue();
+        assertThat(context.awaitCompletion(2, TimeUnit.MINUTES)).isTrue();
     }
 
-    protected void createTestKeyspace(CassandraSidecarTestContext cassandraTestContext)
+    protected void createTestKeyspace()
     {
-        Session session = maybeGetSession(cassandraTestContext);
-
-        session.execute(
-        "CREATE KEYSPACE " + TEST_KEYSPACE +
-        " WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.SimpleStrategy', 'replication_factor': '1' }" +
-        " AND DURABLE_WRITES = true;"
-        );
+        createTestKeyspace(ImmutableMap.of("datacenter1", 1));
     }
 
-    protected QualifiedTableName createTestTable(CassandraSidecarTestContext cassandraTestContext,
-                                                 String createTableStatement)
+    protected void createTestKeyspace(Map<String, Integer> rf)
     {
-        Session session = maybeGetSession(cassandraTestContext);
+        Session session = maybeGetSession();
+        session.execute("CREATE KEYSPACE " + TEST_KEYSPACE +
+                        " WITH REPLICATION = { 'class' : 'NetworkTopologyStrategy', " + generateRfString(rf) + " };");
+    }
+
+    private String generateRfString(Map<String, Integer> dcToRf)
+    {
+        return dcToRf.entrySet().stream().map(e -> String.format("'%s':%d", e.getKey(), e.getValue()))
+                     .collect(Collectors.joining(","));
+    }
+
+    protected QualifiedTableName createTestTable(String createTableStatement)
+    {
+        Session session = maybeGetSession();
         QualifiedTableName tableName = uniqueTestTableFullName();
         session.execute(String.format(createTableStatement, tableName));
         return tableName;
     }
 
-    protected Session maybeGetSession(CassandraSidecarTestContext cassandraTestContext)
+    protected Session maybeGetSession()
     {
-        Session session = cassandraTestContext.session();
+        Session session = sidecarTestContext.session();
         assertThat(session).isNotNull();
         return session;
     }
@@ -148,7 +161,7 @@
 
     public List<Path> findChildFile(CassandraSidecarTestContext context, String hostname, String target)
     {
-        InstanceMetadata instanceConfig = context.getInstancesConfig().instanceFromHost(hostname);
+        InstanceMetadata instanceConfig = context.instancesConfig().instanceFromHost(hostname);
         List<String> parentDirectories = instanceConfig.dataDirs();
 
         return parentDirectories.stream().flatMap(s -> findChildFile(Paths.get(s), target).stream())
@@ -168,4 +181,10 @@
             return Collections.emptyList();
         }
     }
+
+    private void healthCheck(InstancesConfig instancesConfig)
+    {
+        instancesConfig.instances()
+                       .forEach(instanceMetadata -> instanceMetadata.delegate().healthCheck());
+    }
 }
diff --git a/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestModule.java b/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestModule.java
index 9be4ead..523609d 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestModule.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/IntegrationTestModule.java
@@ -18,18 +18,19 @@
 
 package org.apache.cassandra.sidecar;
 
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
 import com.google.inject.AbstractModule;
 import com.google.inject.Provides;
 import com.google.inject.Singleton;
 import org.apache.cassandra.sidecar.cluster.InstancesConfig;
-import org.apache.cassandra.sidecar.config.SSTableUploadConfiguration;
+import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
 import org.apache.cassandra.sidecar.config.ServiceConfiguration;
 import org.apache.cassandra.sidecar.config.SidecarConfiguration;
-import org.apache.cassandra.sidecar.config.ThrottleConfiguration;
-import org.apache.cassandra.sidecar.config.yaml.SSTableUploadConfigurationImpl;
 import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl;
 import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl;
-import org.apache.cassandra.sidecar.config.yaml.ThrottleConfigurationImpl;
 import org.apache.cassandra.sidecar.testing.CassandraSidecarTestContext;
 
 /**
@@ -48,18 +49,58 @@
     @Singleton
     public InstancesConfig instancesConfig()
     {
-        return cassandraTestContext.getInstancesConfig();
+        return new WrapperInstancesConfig(cassandraTestContext);
+    }
+
+    static class WrapperInstancesConfig implements InstancesConfig
+    {
+        private final CassandraSidecarTestContext cassandraTestContext;
+
+        WrapperInstancesConfig(CassandraSidecarTestContext cassandraTestContext)
+        {
+            this.cassandraTestContext = cassandraTestContext;
+        }
+
+        /**
+         * @return metadata of instances owned by the sidecar
+         */
+        public List<InstanceMetadata> instances()
+        {
+            if (cassandraTestContext.isClusterBuilt())
+                return cassandraTestContext.instancesConfig().instances();
+            return Collections.emptyList();
+        }
+
+        /**
+         * Lookup instance metadata by id.
+         *
+         * @param id instance's id
+         * @return instance meta information
+         * @throws NoSuchElementException when the instance with {@code id} does not exist
+         */
+        public InstanceMetadata instanceFromId(int id) throws NoSuchElementException
+        {
+            return cassandraTestContext.instancesConfig().instanceFromId(id);
+        }
+
+        /**
+         * Lookup instance metadata by host name.
+         *
+         * @param host host address of instance
+         * @return instance meta information
+         * @throws NoSuchElementException when the instance for {@code host} does not exist
+         */
+        public InstanceMetadata instanceFromHost(String host) throws NoSuchElementException
+        {
+            return cassandraTestContext.instancesConfig().instanceFromHost(host);
+        }
     }
 
     @Provides
     @Singleton
     public SidecarConfiguration configuration()
     {
-        ThrottleConfiguration throttleConfiguration = new ThrottleConfigurationImpl(1000L);
-        SSTableUploadConfiguration ssTableUploadConfiguration = new SSTableUploadConfigurationImpl(0F);
-        ServiceConfiguration serviceConfiguration = new ServiceConfigurationImpl("127.0.0.1",
-                                                                                 throttleConfiguration,
-                                                                                 ssTableUploadConfiguration);
+        ServiceConfiguration serviceConfiguration = new ServiceConfigurationImpl("127.0.0.1");
         return new SidecarConfigurationImpl(serviceConfiguration);
     }
 }
diff --git a/src/test/integration/org/apache/cassandra/sidecar/common/DelegateTest.java b/src/test/integration/org/apache/cassandra/sidecar/common/DelegateTest.java
index 84cd605..e3665cf 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/common/DelegateTest.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/common/DelegateTest.java
@@ -32,10 +32,10 @@
  */
 class DelegateTest extends IntegrationTestBase
 {
-    @CassandraIntegrationTest
+    @CassandraIntegrationTest(jmx = false)
     void testCorrectVersionIsEnabled()
     {
-        CassandraAdapterDelegate delegate = sidecarTestContext.getInstancesConfig().instances().get(0).delegate();
+        CassandraAdapterDelegate delegate = sidecarTestContext.instancesConfig().instances().get(0).delegate();
         SimpleCassandraVersion version = delegate.version();
         assertThat(version).isNotNull();
         assertThat(version.major).isEqualTo(sidecarTestContext.version.major);
@@ -43,16 +43,16 @@
         assertThat(version).isGreaterThanOrEqualTo(sidecarTestContext.version);
     }
 
-    @CassandraIntegrationTest
+    @CassandraIntegrationTest(jmx = false)
     void testHealthCheck() throws InterruptedException
     {
-        CassandraAdapterDelegate delegate = sidecarTestContext.getInstancesConfig().instances().get(0).delegate();
+        CassandraAdapterDelegate delegate = sidecarTestContext.instancesConfig().instances().get(0).delegate();
 
         delegate.healthCheck();
 
         assertThat(delegate.isUp()).as("health check succeeds").isTrue();
 
-        NodeToolResult nodetoolResult = sidecarTestContext.cluster.get(1).nodetoolResult("disablebinary");
+        NodeToolResult nodetoolResult = sidecarTestContext.cluster().get(1).nodetoolResult("disablebinary");
         assertThat(nodetoolResult.getRc())
         .withFailMessage("Failed to disable binary:\nstdout:" + nodetoolResult.getStdout()
                          + "\nstderr: " + nodetoolResult.getStderr())
@@ -72,7 +72,7 @@
         }
         assertThat(delegate.isUp()).as("health check fails after binary has been disabled").isFalse();
 
-        sidecarTestContext.cluster.get(1).nodetool("enablebinary");
+        sidecarTestContext.cluster().get(1).nodetool("enablebinary");
 
         TimeUnit.SECONDS.sleep(1);
         delegate.healthCheck();
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/RingHandlerIntegrationTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/RingHandlerIntegrationTest.java
index b277a32..38799b6 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/routes/RingHandlerIntegrationTest.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/RingHandlerIntegrationTest.java
@@ -34,8 +34,8 @@
 import org.apache.cassandra.sidecar.IntegrationTestBase;
 import org.apache.cassandra.sidecar.common.data.RingEntry;
 import org.apache.cassandra.sidecar.common.data.RingResponse;
+import org.apache.cassandra.sidecar.testing.CassandraSidecarTestContext;
 import org.apache.cassandra.testing.CassandraIntegrationTest;
-import org.apache.cassandra.testing.CassandraTestContext;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -78,7 +78,7 @@
     @CassandraIntegrationTest
     void retrieveRingWithExistingKeyspace(VertxTestContext context) throws Exception
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
         retrieveRingWithKeyspace(context, TEST_KEYSPACE, response -> {
             assertRingResponseOK(response, sidecarTestContext);
             context.completeNow();
@@ -95,9 +95,9 @@
         });
     }
 
-    void assertRingResponseOK(HttpResponse<Buffer> response, CassandraTestContext cassandraTestContext)
+    void assertRingResponseOK(HttpResponse<Buffer> response, CassandraSidecarTestContext cassandraTestContext)
     {
-        IInstance instance = cassandraTestContext.getCluster().getFirstRunningInstance();
+        IInstance instance = cassandraTestContext.cluster().getFirstRunningInstance();
         IInstanceConfig config = instance.config();
         RingResponse ringResponse = response.bodyAsJson(RingResponse.class);
         assertThat(ringResponse).isNotNull()
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
index c6d7b6e..c7f7203 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/SnapshotsHandlerIntegrationTest.java
@@ -57,7 +57,7 @@
     void createSnapshotEndpointFailsWhenTableDoesNotExist(VertxTestContext context)
     throws InterruptedException
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
 
         WebClient client = WebClient.create(vertx);
         String testRoute = "/api/v1/keyspaces/testkeyspace/tables/non-existent/snapshots/my-snapshot";
@@ -72,7 +72,7 @@
     void createSnapshotFailsWhenSnapshotAlreadyExists(VertxTestContext context)
     throws InterruptedException
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
         String table = createTestTableAndPopulate(sidecarTestContext);
 
         WebClient client = WebClient.create(vertx);
@@ -98,7 +98,7 @@
     void testCreateSnapshotEndpoint(VertxTestContext context)
     throws InterruptedException
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
         String table = createTestTableAndPopulate(sidecarTestContext);
 
         WebClient client = WebClient.create(vertx);
@@ -110,14 +110,12 @@
                   assertThat(response.statusCode()).isEqualTo(OK.code());
 
                   // validate that the snapshot is created
-                  final List<Path> found = findChildFile(sidecarTestContext, "127.0.0.1",
-                                                         "my-snapshot");
-                  assertThat(found).isNotEmpty();
-
-                  assertThat(found.stream().anyMatch(p -> p.endsWith("manifest.json")));
-                  assertThat(found.stream().anyMatch(p -> p.endsWith("schema.cql")));
-                  assertThat(found.stream().anyMatch(p -> p.endsWith("-big-Data.db")));
-
+                  List<Path> found = findChildFile(sidecarTestContext, "127.0.0.1",
+                                                   "my-snapshot");
+                  assertThat(found).isNotEmpty()
+                                   .anyMatch(p -> p.toString().endsWith("manifest.json"))
+                                   .anyMatch(p -> p.toString().endsWith("schema.cql"))
+                                   .anyMatch(p -> p.toString().endsWith("-big-Data.db"));
 
                   context.completeNow();
               })));
@@ -136,7 +134,7 @@
     void deleteSnapshotFailsWhenTableDoesNotExist(VertxTestContext context)
     throws InterruptedException
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
         createTestTableAndPopulate(sidecarTestContext);
 
         String testRoute = "/api/v1/keyspaces/testkeyspace/tables/non-existent/snapshots/my-snapshot";
@@ -147,7 +145,7 @@
     void deleteSnapshotFailsWhenSnapshotDoesNotExist(VertxTestContext context)
     throws InterruptedException
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
         String table = createTestTableAndPopulate(sidecarTestContext);
 
         String testRoute = String.format("/api/v1/keyspaces/%s/tables/%s/snapshots/non-existent",
@@ -160,7 +158,7 @@
     void testDeleteSnapshotEndpoint(VertxTestContext context)
     throws InterruptedException
     {
-        createTestKeyspace(sidecarTestContext);
+        createTestKeyspace();
         String table = createTestTableAndPopulate(sidecarTestContext);
 
         WebClient client = WebClient.create(vertx);
@@ -203,9 +201,9 @@
 
     private String createTestTableAndPopulate(CassandraSidecarTestContext cassandraTestContext)
     {
-        QualifiedTableName tableName = createTestTable(cassandraTestContext,
-                                                       "CREATE TABLE %s (id text PRIMARY KEY, name text);");
-        Session session = maybeGetSession(cassandraTestContext);
+        QualifiedTableName tableName = createTestTable(
+        "CREATE TABLE %s (id text PRIMARY KEY, name text);");
+        Session session = maybeGetSession();
 
         session.execute("INSERT INTO " + tableName + " (id, name) VALUES ('1', 'Francisco');");
         session.execute("INSERT INTO " + tableName + " (id, name) VALUES ('2', 'Saranya');");
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerIntegrationTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerIntegrationTest.java
index 118e575..628c184 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerIntegrationTest.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerIntegrationTest.java
@@ -77,12 +77,12 @@
         // Test the import SSTable endpoint by importing data that was originally truncated.
         // Verify by querying the table contains all the results before truncation and after truncation.
 
-        Session session = maybeGetSession(sidecarTestContext);
-        createTestKeyspace(sidecarTestContext);
+        Session session = maybeGetSession();
+        createTestKeyspace();
         QualifiedTableName tableName = createTestTableAndPopulate(sidecarTestContext, Arrays.asList("a", "b"));
 
         // create a snapshot called <tableName>-snapshot for tbl1
-        UpgradeableCluster cluster = sidecarTestContext.cluster;
+        UpgradeableCluster cluster = sidecarTestContext.cluster();
         final String snapshotStdout = cluster.get(1).nodetoolResult("snapshot",
                                                                     "--tag", tableName.tableName() + "-snapshot",
                                                                     "--table", tableName.tableName(),
@@ -101,7 +101,7 @@
         // verification happens in the host system. When calling import we use the same directory, but the
         // directory does not exist inside the cluster. For that reason we need to do the following to
         // ensure "import" finds the path inside the cluster
-        String uploadStagingDir = sidecarTestContext.getInstancesConfig()
+        String uploadStagingDir = sidecarTestContext.instancesConfig()
                                                     .instanceFromHost("127.0.0.1").stagingDir();
         final String stagingPathInContainer = uploadStagingDir + File.separator + uploadId
                                               + File.separator + tableName.keyspace()
@@ -121,7 +121,7 @@
         }
 
         // Now truncate the contents of the table
-        truncateAndVerify(sidecarTestContext, tableName);
+        truncateAndVerify(tableName);
 
         // Add new data (c, d) to table
         populateTable(session, tableName, Arrays.asList("c", "d"));
@@ -133,7 +133,7 @@
                     () -> client.put(server.actualPort(), "127.0.0.1", testRoute),
                     vertxTestContext.succeeding(response -> vertxTestContext.verify(() -> {
                         assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
-                        assertThat(queryValues(sidecarTestContext, tableName))
+                        assertThat(queryValues(tableName))
                         .containsAll(Arrays.asList("a", "b", "c", "d"));
                         vertxTestContext.completeNow();
                     })));
@@ -159,11 +159,10 @@
                        })));
     }
 
-    private void truncateAndVerify(CassandraSidecarTestContext cassandraTestContext,
-                                   QualifiedTableName qualifiedTableName)
+    private void truncateAndVerify(QualifiedTableName qualifiedTableName)
     throws InterruptedException
     {
-        Session session = maybeGetSession(cassandraTestContext);
+        Session session = maybeGetSession();
         session.execute("TRUNCATE TABLE " + qualifiedTableName);
 
         while (true)
@@ -175,9 +174,9 @@
         }
     }
 
-    private List<String> queryValues(CassandraSidecarTestContext cassandraTestContext, QualifiedTableName tableName)
+    private List<String> queryValues(QualifiedTableName tableName)
     {
-        Session session = maybeGetSession(cassandraTestContext);
+        Session session = maybeGetSession();
         return session.execute("SELECT id FROM " + tableName)
                       .all()
                       .stream()
@@ -188,9 +187,9 @@
     private QualifiedTableName createTestTableAndPopulate(CassandraSidecarTestContext cassandraTestContext,
                                                           List<String> values)
     {
-        QualifiedTableName tableName = createTestTable(cassandraTestContext,
-                                                       "CREATE TABLE IF NOT EXISTS %s (id text, PRIMARY KEY(id));");
-        Session session = maybeGetSession(cassandraTestContext);
+        QualifiedTableName tableName = createTestTable(
+        "CREATE TABLE IF NOT EXISTS %s (id text, PRIMARY KEY(id));");
+        Session session = maybeGetSession();
         populateTable(session, tableName, values);
         return tableName;
     }
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BaseTokenRangeIntegrationTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BaseTokenRangeIntegrationTest.java
new file mode 100644
index 0000000..0d789c3
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BaseTokenRangeIntegrationTest.java
@@ -0,0 +1,287 @@
+/*
+ * 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.tokenrange;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+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 the in-jvm dtest framework.
+ */
+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).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(BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws IOException
+    {
+        return getMultiDCCluster(initializer, cassandraTestContext, null);
+    }
+
+    protected UpgradeableCluster getMultiDCCluster(BiConsumer<ClassLoader, Integer> initializer,
+                                                   ConfigurableCassandraTestContext cassandraTestContext,
+                                                   Consumer<UpgradeableCluster.Builder> additionalConfigurator)
+    throws IOException
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier mdcTokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                   annotation.newNodesPerDc(),
+                                                                                   annotation.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")));
+
+            if (additionalConfigurator != null)
+            {
+                additionalConfigurator.accept(builder);
+            }
+        });
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges()
+    {
+        return generateExpectedRanges(true);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(boolean isCrossDCKeyspace)
+    {
+
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        // For single DC keyspaces, the ranges are initially allocated replicas from both DCs. As a result,
+        // we will take into account the node count across all DCs. It is only while accounting for the new/joining
+        // nodes that we will limit the nodes to the single DC, as the pending nodes for the given keyspace will
+        // exclude the nodes from other DCs.
+
+        int nodeCount = isCrossDCKeyspace ?
+                        (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs() :
+                        (annotation.nodesPerDc() * annotation.numDcs()) + annotation.newNodesPerDc();
+
+        return generateExpectedRanges(nodeCount);
+    }
+
+    protected List<Range<BigInteger>> generateExpectedRanges(int nodeCount)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                annotation.newNodesPerDc(),
+                                                                                annotation.numDcs(),
+                                                                                1);
+
+        TreeSet<BigInteger> tokens = new TreeSet<>();
+        int node = 1;
+        while (node <= nodeCount)
+        {
+            tokens.add(new BigInteger(tokenSupplier.tokens(node++).stream().findFirst().get()));
+        }
+
+        List<Range<BigInteger>> expectedRanges = new ArrayList<>();
+        BigInteger startToken = Partitioner.Murmur3.minToken;
+        BigInteger endToken = Partitioner.Murmur3.maxToken;
+
+        BigInteger prevToken = tokens.pollFirst();
+        Range<BigInteger> firstRange = Range.openClosed(startToken, prevToken);
+        expectedRanges.add(firstRange);
+
+        for (BigInteger token : tokens)
+        {
+            BigInteger currentToken = token;
+            expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            prevToken = currentToken;
+        }
+
+        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)
+    {
+        validateWriteReplicaMappings(writeReplicas, expectedRangeMapping, true);
+    }
+
+    protected void validateWriteReplicaMappings(List<TokenRangeReplicasResponse.ReplicaInfo> writeReplicas,
+                                                Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMapping,
+                                                boolean isCrossDCKeyspace)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        // Validates the no. of ranges in the write-replica mappings match the no. of expected ranges
+        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
+            List<String> replicaSetNoPort = r.replicasByDatacenter().get("datacenter1")
+                                             .stream()
+                                             .map(node -> node.split(":")[0])
+                                             .collect(Collectors.toList());
+            assertThat(replicaSetNoPort)
+            .containsExactlyInAnyOrderElementsOf(expectedRangeMapping.get("datacenter1").get(range));
+
+            if (annotation.numDcs() > 1 && isCrossDCKeyspace)
+            {
+                assertThat(expectedRangeMapping).containsKey("datacenter2");
+                assertThat(expectedRangeMapping.get("datacenter2")).containsKey(range);
+
+                List<String> replicaSetNoPortDc2 = r.replicasByDatacenter().get("datacenter2")
+                                                    .stream()
+                                                    .map(node -> node.split(":")[0])
+                                                    .collect(Collectors.toList());
+                assertThat(replicaSetNoPortDc2)
+                .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,
+                                 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);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestMultiDCRf3.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestMultiDCRf3.java
new file mode 100644
index 0000000..8e70364
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestMultiDCRf3.java
@@ -0,0 +1,59 @@
+/*
+ * 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.tokenrange;
+
+import java.util.Arrays;
+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 the in-jvm dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+class BasicTestMultiDCRf3 extends BaseTokenRangeIntegrationTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, numDcs = 2)
+    void retrieveMappingMultiDcRf3(VertxTestContext context) throws Exception
+    {
+        int replicationFactor = 3;
+        createTestKeyspace(ImmutableMap.of("replication_factor", replicationFactor));
+        retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+            TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+            // the keyspace is replicated to both DCs
+            assertMappingResponseOK(mappingResponse,
+                                    replicationFactor,
+                                    Sets.newHashSet(Arrays.asList("datacenter1", "datacenter2")));
+            context.completeNow();
+        });
+        assertThat(context.awaitCompletion(30, TimeUnit.SECONDS)).isTrue();
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestMultiDCSingleReplicated.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestMultiDCSingleReplicated.java
new file mode 100644
index 0000000..782e760
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestMultiDCSingleReplicated.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.routes.tokenrange;
+
+import java.util.Collections;
+
+import com.google.common.collect.ImmutableMap;
+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;
+
+/**
+ * Test the token range replica mapping endpoint with the in-jvm dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+class BasicTestMultiDCSingleReplicated extends BaseTokenRangeIntegrationTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, numDcs = 2)
+    void retrieveMappingSingleDCReplicatedRf3(VertxTestContext context)
+    throws Exception
+    {
+        int replicationFactor = 3;
+        createTestKeyspace(ImmutableMap.of("datacenter1", replicationFactor));
+        retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+            TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+            assertMappingResponseOK(mappingResponse, replicationFactor, Collections.singleton("datacenter1"));
+            context.completeNow();
+        });
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestRf1.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestRf1.java
new file mode 100644
index 0000000..b11bcd0
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestRf1.java
@@ -0,0 +1,53 @@
+/*
+ * 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.tokenrange;
+
+import java.util.Collections;
+
+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.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 the in-jvm dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+class BasicTestRf1 extends BaseTokenRangeIntegrationTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 3)
+    void retrieveMappingRf1(VertxTestContext context) throws Exception
+    {
+        createTestKeyspace();
+        retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+            assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
+            TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+            assertMappingResponseOK(mappingResponse, 1, Collections.singleton("datacenter1"));
+            context.completeNow();
+        });
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestRf3.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestRf3.java
new file mode 100644
index 0000000..b99e6fd
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestRf3.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tokenrange;
+
+import java.util.Collections;
+
+import com.google.common.collect.ImmutableMap;
+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;
+
+/**
+ * Test the token range replica mapping endpoint with the in-jvm dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+class BasicTestRf3 extends BaseTokenRangeIntegrationTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5)
+    void retrieveMappingRf3(VertxTestContext context) throws Exception
+    {
+        int replicationFactor = 3;
+        createTestKeyspace(ImmutableMap.of("datacenter1", replicationFactor));
+        retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+            TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+            assertMappingResponseOK(mappingResponse, replicationFactor, Collections.singleton("datacenter1"));
+            context.completeNow();
+        });
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestUnknownKeyspace.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestUnknownKeyspace.java
new file mode 100644
index 0000000..2b1a158
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/BasicTestUnknownKeyspace.java
@@ -0,0 +1,53 @@
+/*
+ * 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.tokenrange;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.json.JsonObject;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test the token range replica mapping endpoint with the in-jvm dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+class BasicTestUnknownKeyspace extends BaseTokenRangeIntegrationTest
+{
+    @CassandraIntegrationTest
+    void retrieveMappingWithUnknownKeyspace(VertxTestContext context) throws Exception
+    {
+        retrieveMappingWithKeyspace(context, "unknown_ks", response -> {
+            int errorCode = HttpResponseStatus.NOT_FOUND.code();
+            assertThat(response.statusCode()).isEqualTo(errorCode);
+            JsonObject body = response.bodyAsJsonObject();
+            assertThat(body.getInteger("code")).isEqualTo(errorCode);
+            assertThat(body.getString("message")).contains("Unknown keyspace");
+
+            context.completeNow();
+        });
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningBaseTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningBaseTest.java
new file mode 100644
index 0000000..924eb71
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningBaseTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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.tokenrange;
+
+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.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+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 io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.Feature;
+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 static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Base class for TokenRangeIntegrationJoining tests
+ */
+class JoiningBaseTest extends BaseTokenRangeIntegrationTest
+{
+    void runJoiningTestScenario(VertxTestContext context,
+                                CountDownLatch transientStateStart,
+                                CountDownLatch transientStateEnd,
+                                UpgradeableCluster cluster,
+                                List<Range<BigInteger>> expectedRanges,
+                                Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings,
+                                boolean isCrossDCKeyspace)
+    throws Exception
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        try
+        {
+            Set<String> dcReplication;
+            if (annotation.numDcs() > 1 && isCrossDCKeyspace)
+            {
+                createTestKeyspace(ImmutableMap.of("replication_factor", DEFAULT_RF));
+                dcReplication = Sets.newHashSet(Arrays.asList("datacenter1", "datacenter2"));
+            }
+            else
+            {
+                createTestKeyspace(ImmutableMap.of("datacenter1", DEFAULT_RF));
+                dcReplication = Collections.singleton("datacenter1");
+            }
+
+            IUpgradeableInstance seed = cluster.get(1);
+
+            List<IUpgradeableInstance> newInstances = new ArrayList<>();
+            // Go over new nodes and add them once for each DC
+            for (int i = 0; i < annotation.newNodesPerDc(); i++)
+            {
+                int dcNodeIdx = 1; // Use node 2's DC
+                for (int dc = 1; dc <= annotation.numDcs(); dc++)
+                {
+                    IUpgradeableInstance dcNode = cluster.get(dcNodeIdx++);
+                    IUpgradeableInstance newInstance = ClusterUtils.addInstance(cluster,
+                                                                                dcNode.config().localDatacenter(),
+                                                                                dcNode.config().localRack(),
+                                                                                inst -> {
+                                                                                    inst.set("auto_bootstrap", true);
+                                                                                    inst.with(Feature.GOSSIP,
+                                                                                              Feature.JMX,
+                                                                                              Feature.NATIVE_PROTOCOL);
+                                                                                });
+                    new Thread(() -> newInstance.startup(cluster)).start();
+                    newInstances.add(newInstance);
+                }
+            }
+
+            Uninterruptibles.awaitUninterruptibly(transientStateStart, 2, TimeUnit.MINUTES);
+
+            for (IUpgradeableInstance newInstance : newInstances)
+            {
+                ClusterUtils.awaitRingState(seed, newInstance, "Joining");
+            }
+
+            retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+                assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
+                TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+                assertMappingResponseOK(mappingResponse,
+                                        DEFAULT_RF,
+                                        dcReplication);
+                int finalNodeCount = (annotation.nodesPerDc() + annotation.newNodesPerDc()) * annotation.numDcs();
+                TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                        annotation.newNodesPerDc(),
+                                                                                        annotation.numDcs(),
+                                                                                        1);
+                // New split ranges resulting from joining nodes and corresponding tokens
+                List<Range<BigInteger>> splitRanges = extractSplitRanges(annotation.newNodesPerDc() *
+                                                                         annotation.numDcs(),
+                                                                         finalNodeCount,
+                                                                         tokenSupplier,
+                                                                         expectedRanges);
+
+                List<Integer> newNodes = newInstances.stream().map(i -> i.config().num()).collect(Collectors.toList());
+                validateNodeStates(mappingResponse,
+                                   dcReplication,
+                                   nodeNumber -> newNodes.contains(nodeNumber) ? "Joining" : "Normal");
+
+                validateTokenRanges(mappingResponse, expectedRanges);
+                validateReplicaMapping(mappingResponse,
+                                       newInstances,
+                                       isCrossDCKeyspace,
+                                       splitRanges,
+                                       expectedRangeMappings);
+
+                context.completeNow();
+            });
+        }
+        finally
+        {
+            for (int i = 0;
+                 i < (annotation.newNodesPerDc() * annotation.numDcs()); i++)
+            {
+                transientStateEnd.countDown();
+            }
+        }
+    }
+
+    private void validateReplicaMapping(TokenRangeReplicasResponse mappingResponse,
+                                        List<IUpgradeableInstance> newInstances,
+                                        boolean isCrossDCKeyspace,
+                                        List<Range<BigInteger>> splitRanges,
+                                        Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    {
+
+        if (!isCrossDCKeyspace)
+        {
+            newInstances = newInstances.stream()
+                                       .filter(i -> i.config().localDatacenter().equals("datacenter1"))
+                                       .collect(Collectors.toList());
+        }
+
+        List<String> transientNodeAddresses = newInstances.stream().map(i -> {
+            InetSocketAddress address = i.config().broadcastAddress();
+            return address.getAddress().getHostAddress() +
+                   ":" +
+                   address.getPort();
+        }).collect(Collectors.toList());
+
+        Set<String> writeReplicaInstances = instancesFromReplicaSet(mappingResponse.writeReplicas());
+        Set<String> readReplicaInstances = instancesFromReplicaSet(mappingResponse.readReplicas());
+
+        Set<String> splitRangeReplicas
+        = mappingResponse.writeReplicas().stream()
+                         .filter(w -> matchSplitRanges(w, splitRanges))
+                         .map(r ->
+                              r.replicasByDatacenter().values())
+                         .flatMap(Collection::stream)
+                         .flatMap(list -> list.stream())
+                         .collect(Collectors.toSet());
+
+        assertThat(readReplicaInstances).doesNotContainAnyElementsOf(transientNodeAddresses);
+        // Validate that the new nodes are mapped to the split ranges
+        assertThat(splitRangeReplicas).containsAll(transientNodeAddresses);
+        assertThat(writeReplicaInstances).containsAll(transientNodeAddresses);
+
+        validateWriteReplicaMappings(mappingResponse.writeReplicas(), expectedRangeMappings, isCrossDCKeyspace);
+    }
+
+    private List<Range<BigInteger>> extractSplitRanges(int newNodes,
+                                                       int finalNodeCount,
+                                                       TokenSupplier tokenSupplier,
+                                                       List<Range<BigInteger>> expectedRanges)
+    {
+
+        int newNode = 1;
+        List<BigInteger> newNodeTokens = new ArrayList<>();
+        while (newNode <= newNodes)
+        {
+            int nodeIdx = finalNodeCount - newNode;
+            newNodeTokens.add(new BigInteger(tokenSupplier.tokens(nodeIdx).stream().findFirst().get()));
+            newNode++;
+        }
+
+        return expectedRanges.stream()
+                             .filter(r -> newNodeTokens.contains(r.upperEndpoint()) ||
+                                          newNodeTokens.contains(r.lowerEndpoint()))
+                             .collect(Collectors.toList());
+    }
+
+    private boolean matchSplitRanges(TokenRangeReplicasResponse.ReplicaInfo range,
+                                     List<Range<BigInteger>> expectedSplitRanges)
+    {
+        return expectedSplitRanges.stream()
+                                  .anyMatch(s -> range.start().equals(s.lowerEndpoint().toString()) &&
+                                                 range.end().equals(s.upperEndpoint().toString()));
+    }
+
+    void runJoiningTestScenario(VertxTestContext context,
+                                ConfigurableCassandraTestContext cassandraTestContext,
+                                BiConsumer<ClassLoader, Integer> instanceInitializer,
+                                CountDownLatch transientStateStart,
+                                CountDownLatch transientStateEnd,
+                                Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    throws Exception
+    {
+
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                annotation.newNodesPerDc(),
+                                                                                annotation.numDcs(),
+                                                                                1);
+
+        UpgradeableCluster cluster = cassandraTestContext.configureAndStartCluster(builder -> {
+            builder.withInstanceInitializer(instanceInitializer);
+            builder.withTokenSupplier(tokenSupplier);
+        });
+
+        runJoiningTestScenario(context,
+                               transientStateStart,
+                               transientStateEnd,
+                               cluster,
+                               generateExpectedRanges(),
+                               expectedRangeMappings,
+                               true);
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTest.java
new file mode 100644
index 0000000..49335ac
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.tokenrange;
+
+import java.util.Collections;
+
+import com.google.common.collect.ImmutableMap;
+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.Feature;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.ClusterUtils;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Cluster expansion scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+public class JoiningTest extends JoiningBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 3, newNodesPerDc = 1, gossip = true, network = true)
+    void retrieveMappingWithKeyspaceWithAddNode(VertxTestContext context) throws Exception
+    {
+        createTestKeyspace(ImmutableMap.of("replication_factor", DEFAULT_RF));
+        UpgradeableCluster cluster = sidecarTestContext.cluster();
+        IUpgradeableInstance instance = cluster.get(1);
+        IUpgradeableInstance newInstance = ClusterUtils.addInstance(cluster,
+                                                                    instance.config().localDatacenter(),
+                                                                    instance.config().localRack(),
+                                                                    inst -> inst.with(Feature.NETWORK,
+                                                                                      Feature.GOSSIP,
+                                                                                      Feature.JMX,
+                                                                                      Feature.NATIVE_PROTOCOL));
+        cluster.get(4).startup(cluster);
+        ClusterUtils.awaitRingState(instance, newInstance, "Normal");
+
+        retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+            TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+            assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
+            assertMappingResponseOK(mappingResponse, DEFAULT_RF, Collections.singleton("datacenter1"));
+            context.completeNow();
+        });
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestDoubleCluster.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestDoubleCluster.java
new file mode 100644
index 0000000..65b00fb
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestDoubleCluster.java
@@ -0,0 +1,217 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Cluster expansion scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+public class JoiningTestDoubleCluster extends JoiningBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithDoubleClusterSize(VertxTestContext context,
+                                              ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperDoubleClusterSize.reset();
+        runJoiningTestScenario(context,
+                               cassandraTestContext,
+                               BBHelperDoubleClusterSize::install,
+                               BBHelperDoubleClusterSize.transientStateStart,
+                               BBHelperDoubleClusterSize.transientStateEnd,
+                               generateExpectedRangeMappingDoubleClusterSize());
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * doubling in size
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * We generate the expected ranges by using
+     * 1) the initial token allocations to nodes (prior to adding nodes) shown under "Initial Ranges"
+     * (in the comment block below),
+     * 2)the "pending node ranges" and
+     * 3) the final token allocations per node.
+     * <p>
+     * Step 1: Prepare ranges starting from partitioner min-token, ending at partitioner max-token using (3) above
+     * Step 2: Create the cascading list of replica-sets based on the RF (3) for each range using the initial node list
+     * Step 3: Add replicas to ranges based on (1) and (2) above.
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingDoubleClusterSize()
+    {
+
+        /*
+         *
+         * Initial Ranges:
+         * [-9223372036854775808, -6148914691236517205]:["127.0.0.3","127.0.0.2","127.0.0.1"]
+         * [-6148914691236517205, -3074457345618258603]:["127.0.0.3","127.0.0.2","127.0.0.4"]
+         * [-3074457345618258603, -1]:["127.0.0.3","127.0.0.5","127.0.0.4"]
+         * [-1, 3074457345618258601]:["127.0.0.5","127.0.0.4","127.0.0.1"]
+         * [3074457345618258601, 6148914691236517203]:["127.0.0.5","127.0.0.2","127.0.0.1"]
+         * [6148914691236517203, 9223372036854775807]:["127.0.0.3","127.0.0.2","127.0.0.1"]
+         *
+         * New node tokens
+         * 127.0.0.6 at token -4611686018427387904
+         * 127.0.0.7 at token -1537228672809129302
+         * 127.0.0.8 at token 1537228672809129300
+         * 127.0.0.9 at token 4611686018427387902
+         * 127.0.0.10 at token 7686143364045646504
+         *
+         * Pending Ranges:
+         * [-3074457345618258603, -1]=[127.0.0.9:64060, 127.0.0.8:64055]
+         * [6148914691236517203, -6148914691236517205]=[127.0.0.6:64047, 127.0.0.7:64050] (wrap-around)
+         * [-6148914691236517205, -4611686018427387904]=[127.0.0.6:64047]
+         * [6148914691236517203, 7686143364045646504]=[127.0.0.10:64068]
+         * [-3074457345618258603, -1537228672809129302]=[127.0.0.7:64050]
+         * [3074457345618258601, 6148914691236517203]=[127.0.0.6:64047, 127.0.0.10:64068]
+         * [-1, 1537228672809129300]=[127.0.0.8:64055]
+         * [-6148914691236517205, -3074457345618258603]=[127.0.0.7:64050, 127.0.0.8:64055]
+         * [-1, 3074457345618258601]=[127.0.0.9:64060, 127.0.0.10:64068]
+         * [3074457345618258601, 4611686018427387902]=[127.0.0.9:64060]
+         *
+         */
+
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        // [-9223372036854775808, -6148914691236517205]
+        mapping.put(expectedRanges.get(0),
+                    Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.6", "127.0.0.7"));
+        // [-6148914691236517205, -4611686018427387904]
+        mapping.put(expectedRanges.get(1),
+                    Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.6", "127.0.0.7", "127.0.0.8"));
+        // [-4611686018427387904, -3074457345618258603]
+        mapping.put(expectedRanges.get(2),
+                    Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.7", "127.0.0.8"));
+        // [-3074457345618258603, -1537228672809129302]
+        mapping.put(expectedRanges.get(3),
+                    Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.7", "127.0.0.8", "127.0.0.9"));
+        // [-1537228672809129302, -1]
+        mapping.put(expectedRanges.get(4),
+                    Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.8", "127.0.0.9"));
+        // [-1, 1537228672809129300]
+        mapping.put(expectedRanges.get(5),
+                    Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1", "127.0.0.8", "127.0.0.9", "127.0.0.10"));
+        // [1537228672809129300, 3074457345618258601]
+        mapping.put(expectedRanges.get(6),
+                    Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1", "127.0.0.9", "127.0.0.10"));
+        // [3074457345618258601, 4611686018427387902]
+        mapping.put(expectedRanges.get(7),
+                    Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2", "127.0.0.6", "127.0.0.9", "127.0.0.10"));
+        // [4611686018427387902, 6148914691236517203]
+        mapping.put(expectedRanges.get(8),
+                    Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2", "127.0.0.6", "127.0.0.10"));
+        // [6148914691236517203, 7686143364045646504]
+        mapping.put(expectedRanges.get(9),
+                    Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.6", "127.0.0.7", "127.0.0.10"));
+        // Un-wrapped wrap-around range with the nodes in the initial range
+        // [7686143364045646504, 9223372036854775807]
+        mapping.put(expectedRanges.get(10),
+                    Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.6", "127.0.0.7"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for doubling cluster size
+     */
+    @Shared
+    public static class BBHelperDoubleClusterSize
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(5);
+        static CountDownLatch transientStateEnd = 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
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(5);
+            transientStateEnd = new CountDownLatch(5);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultiDC.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultiDC.java
new file mode 100644
index 0000000..a5e603a
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultiDC.java
@@ -0,0 +1,273 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Multi-DC Cluster expansion scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+public class JoiningTestMultiDC extends JoiningBaseTest
+{
+    @CassandraIntegrationTest(
+    nodesPerDc = 3, newNodesPerDc = 3, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingsDoubleClusterSizeMultiDC(VertxTestContext context,
+                                                  ConfigurableCassandraTestContext cassandraTestContext)
+    throws Exception
+    {
+        BBHelperDoubleClusterMultiDC.reset();
+        UpgradeableCluster cluster = getMultiDCCluster(BBHelperDoubleClusterMultiDC::install, cassandraTestContext);
+
+        runJoiningTestScenario(context,
+                               BBHelperDoubleClusterMultiDC.transientStateStart,
+                               BBHelperDoubleClusterMultiDC.transientStateEnd,
+                               cluster,
+                               generateExpectedRanges(),
+                               generateExpectedRangeDoubleClusterSizeMultiDC(),
+                               true);
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 6 node cluster
+     * across 2 DCs with the 6 nodes joining the cluster (3 per DC)
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * In a multi-DC scenario, a single range will have nodes from both DCs. The replicas are grouped by DC here
+     * to allow per-DC validation as returned from the sidecar endpoint.
+     * <p>
+     * We generate the expected ranges by using
+     * 1) the initial token allocations to nodes (prior to adding nodes) shown under "Initial Ranges"
+     * (in the comment block below),
+     * 2)the "pending node ranges" and
+     * 3) the final token allocations per node.
+     * <p>
+     * Step 1: Prepare ranges starting from partitioner min-token, ending at partitioner max-token using (3) above
+     * Step 2: Create the cascading list of replica-sets based on the RF (3) for each range using the initial node list
+     * Step 3: Add replicas to ranges based on (1) and (2) above
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeDoubleClusterSizeMultiDC()
+    {
+        /*
+         * Initial Ranges:
+         * [-9223372036854775808", "-4611686018427387907"]:["127.0.0.3","127.0.0.5","127.0.0.1",
+         *                                                  "127.0.0.6","127.0.0.2","127.0.0.4"]
+         * [-4611686018427387907", "-4611686018427387906"]:["127.0.0.3","127.0.0.5","127.0.0.1",
+         *                                                  "127.0.0.6","127.0.0.2","127.0.0.4"]
+         * [-4611686018427387906", "-7"]:["127.0.0.3","127.0.0.5","127.0.0.1",
+         *                                                  "127.0.0.6","127.0.0.2","127.0.0.4"]
+         * [-7", "-6"]:["127.0.0.5","127.0.0.3","127.0.0.1",
+         *                                                  "127.0.0.6","127.0.0.2","127.0.0.4"]
+         * [-6", "4611686018427387893"]:["127.0.0.5","127.0.0.3","127.0.0.1", "127.0.0.6",
+         *                                                  "127.0.0.2","127.0.0.4"]
+         * [4611686018427387893", "4611686018427387894"]:["127.0.0.3","127.0.0.5","127.0.0.1", "127.0.0.6","127.0.0.2",
+         *                                                  "127.0.0.4"]
+         * ["4611686018427387894"", "9223372036854775807"]:["127.0.0.3","127.0.0.5","127.0.0.1", "127.0.0.6",
+         *                                                  "127.0.0.2","127.0.0.4:]
+         *
+         *  New Node tokens:
+         * 127.0.0.7 at token -2305843009213693956
+         * 127.0.0.8 at token -2305843009213693955
+         * 127.0.0.9 at token 2305843009213693944
+         * 127.0.0.10 at token 2305843009213693945
+         * 127.0.0.11 at token 6917529027641081844
+         * 127.0.0.12 at token 6917529027641081845
+         *
+         *
+         * Pending Ranges:
+         * [-6, 2305843009213693944]=[127.0.0.9:62801]
+         * [-6, 2305843009213693945]=[127.0.0.10:62802]
+         * [-6, 4611686018427387893]=[127.0.0.12:62804, 127.0.0.7:62799, 127.0.0.8:62800, 127.0.0.11:62803]
+         * [4611686018427387894, -4611686018427387907]=[127.0.0.7:62799, 127.0.0.8:62800, 127.0.0.9:62801,
+         * 127.0.0.10:62802] (wrap-around)
+         * [-4611686018427387906, -2305843009213693956]=[127.0.0.7:62799]
+         * [-4611686018427387907, -4611686018427387906]=[127.0.0.7:62799, 127.0.0.8:62800, 127.0.0.9:62801,
+         * 127.0.0.10:62802, 127.0.0.11:62803]
+         * [-4611686018427387906, -7]=[127.0.0.12:62804, 127.0.0.9:62801, 127.0.0.10:62802, 127.0.0.11:62803]
+         * [-4611686018427387906, -2305843009213693955]=[127.0.0.8:62800]
+         * [4611686018427387894, 6917529027641081844]=[127.0.0.11:62803]
+         * [4611686018427387894, 6917529027641081845]=[127.0.0.12:62804]
+         * [4611686018427387893, 4611686018427387894]=[127.0.0.12:62804, 127.0.0.7:62799, 127.0.0.8:62800,
+         * 127.0.0.9:62801, 127.0.0.11:62803]
+         *
+         */
+
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> dc1Mapping = new HashMap<>();
+        Map<Range<BigInteger>, List<String>> dc2Mapping = new HashMap<>();
+
+        dc1Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6", "127.0.0.8",
+                                                            "127.0.0.10"));
+
+        dc1Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.1", "127.0.0.7",
+                                                            "127.0.0.9", "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6", "127.0.0.8",
+                                                            "127.0.0.10"));
+
+
+        dc1Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.1", "127.0.0.7",
+                                                            "127.0.0.9", "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6", "127.0.0.8",
+                                                            "127.0.0.10", "127.0.0.12"));
+
+
+        dc1Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.3", "127.0.0.9",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.2", "127.0.0.10",
+                                                            "127.0.0.8", "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.3", "127.0.0.9",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.2", "127.0.0.10",
+                                                            "127.0.0.12"));
+        dc1Mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.9", "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.6", "127.0.0.2", "127.0.0.4", "127.0.0.10",
+                                                            "127.0.0.12"));
+
+
+        dc1Mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.9", "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.6", "127.0.0.2", "127.0.0.4", "127.0.0.8",
+                                                            "127.0.0.10", "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.6", "127.0.0.2", "127.0.0.4", "127.0.0.8",
+                                                            "127.0.0.10", "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.7", "127.0.0.11", "127.0.0.1", "127.0.0.3",
+                                                            "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.6", "127.0.0.2", "127.0.0.4", "127.0.0.8",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.9", "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.8", "127.0.0.12", "127.0.0.2", "127.0.0.6",
+                                                            "127.0.0.4"));
+
+        dc1Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                             "127.0.0.9", "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.2", "127.0.0.10",
+                                                             "127.0.0.8", "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                             "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.2", "127.0.0.10",
+                                                             "127.0.0.8", "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(12), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                             "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(12), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.2", "127.0.0.10",
+                                                             "127.0.0.8"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", dc1Mapping);
+                put("datacenter2", dc2Mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for multiple joining nodes
+     */
+    @Shared
+    public static class BBHelperDoubleClusterMultiDC
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(6);
+        static CountDownLatch transientStateEnd = new CountDownLatch(6);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves doubling the size of a 6 node cluster (3 per DC)
+            // We intercept the bootstrap of nodes (7-12) 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(BBHelperDoubleClusterMultiDC.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
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(6);
+            transientStateEnd = new CountDownLatch(6);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultiDCSingleReplicated.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultiDCSingleReplicated.java
new file mode 100644
index 0000000..db24057
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultiDCSingleReplicated.java
@@ -0,0 +1,202 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Multi-DC Cluster expansion scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+public class JoiningTestMultiDCSingleReplicated extends JoiningBaseTest
+{
+    @CassandraIntegrationTest(
+    nodesPerDc = 5, newNodesPerDc = 1, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingsSingleDCReplicatedKeyspace(VertxTestContext context,
+                                                    ConfigurableCassandraTestContext cassandraTestContext)
+    throws Exception
+    {
+        BBHelperMultiDC.reset();
+        UpgradeableCluster cluster = getMultiDCCluster(BBHelperMultiDC::install, cassandraTestContext);
+
+        runJoiningTestScenario(context,
+                               BBHelperMultiDC.transientStateStart,
+                               BBHelperMultiDC.transientStateEnd,
+                               cluster,
+                               generateExpectedRanges(false),
+                               generateExpectedRangeMappingOneof2DCs(),
+                               false);
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * with the additional node joining the cluster
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * Ranges that include the joining node will have [RF + no. joining nodes in replica-set] replicas with
+     * the replicas being the existing nodes in ring-order.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with E being the joining node)
+     * Expected Range 2 - B, C, D, E
+     */
+    private HashMap<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingOneof2DCs()
+    {
+
+        /*
+         * Initial Ranges:
+         * [-9223372036854775808, -5869418568907584607]=[127.0.0.1:52914, 127.0.0.3:52916, 127.0.0.5:52918]
+         * [-5869418568907584607, -5869418568907584606]=[127.0.0.3:52916, 127.0.0.5:52918, 127.0.0.7:52920]
+         * [-5869418568907584606, -2515465100960393407]=[127.0.0.3:52918, 127.0.0.5:52920, 127.0.0.7:52922]
+         * [-2515465100960393407, -2515465100960393406]=[127.0.0.5:52918, 127.0.0.7:52920, 127.0.0.9:52922]
+         * [-2515465100960393406, 838488366986797793]=[127.0.0.5:52918, 127.0.0.7:52920, 127.0.0.9:52922]
+         * [838488366986797793, 838488366986797794]=[127.0.0.7:52920, 127.0.0.9:52922, 127.0.0.1:52914]
+         * [838488366986797794, 4192441834933988993]=[127.0.0.7:52920, 127.0.0.9:52922, 127.0.0.1:52914]
+         * [4192441834933988993, 4192441834933988994]=[127.0.0.9:52922, 127.0.0.1:52914, 127.0.0.3:52916]
+         * [4192441834933988994, 7546395302881180193]=[127.0.0.9:52922, 127.0.0.1:52914, 127.0.0.3:52916]
+         * [7546395302881180193, 7546395302881180194]=[127.0.0.1:52914, 127.0.0.3:52916, 127.0.0.5:52918]
+         * [7546395302881180194, 9223372036854775807]=[127.0.0.1:52914, 127.0.0.3:52916, 127.0.0.5:52918]
+         *
+         * Pending Ranges:
+         * [-5869418568907584607, -5869418568907584606]=[127.0.0.11:58400]
+         * [-5869418568907584606, -4192441834933989006]=[127.0.0.11:58400]
+         * [4192441834933988993, 4192441834933988994]=[127.0.0.11:58400]
+         * [7546395302881180194, -5869418568907584607]=[127.0.0.11:58400] (wrap-around)
+         * [7546395302881180193, 7546395302881180194]=[127.0.0.11:58400]
+         * [4192441834933988994, 7546395302881180193]=[127.0.0.11:58400]
+         *
+         */
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges(false);
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        // [-9223372036854775808, -5869418568907584607]
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.11"));
+        // [-5869418568907584607, -5869418568907584606]
+        mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7", "127.0.0.11"));
+        // [-5869418568907584606, -4192441834933989006]
+        mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7", "127.0.0.11"));
+        // [-4192441834933989006, -2515465100960393407]
+        mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7"));
+        // [-2515465100960393407, -2515465100960393406]
+        mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9"));
+        // [-2515465100960393406, 838488366986797793]
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9"));
+        // [838488366986797793, 838488366986797794]
+        mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        // [838488366986797794, 4192441834933988993]
+        mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        // [4192441834933988993, 4192441834933988994]
+        mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3", "127.0.0.11"));
+        // [4192441834933988994, 7546395302881180193]
+        mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3", "127.0.0.11"));
+        // [7546395302881180193, 7546395302881180194]
+        mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.11"));
+        // [7546395302881180194, 9223372036854775807]
+        mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5", "127.0.0.11"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for multiple joining nodes
+     */
+    @Shared
+    public static class BBHelperMultiDC
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(2);
+        static CountDownLatch transientStateEnd = new CountDownLatch(2);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves adding 2 nodes to a 10 node cluster (5 per DC)
+            // We intercept the bootstrap of nodes (11,12) to validate token ranges
+            if (nodeNumber > 10)
+            {
+                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(BBHelperMultiDC.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
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(2);
+            transientStateEnd = new CountDownLatch(2);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultipleNodes.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultipleNodes.java
new file mode 100644
index 0000000..65936f8
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestMultipleNodes.java
@@ -0,0 +1,189 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Cluster expansion scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+public class JoiningTestMultipleNodes extends JoiningBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 3, newNodesPerDc = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithMultipleJoiningNodes(VertxTestContext context,
+                                                 ConfigurableCassandraTestContext cassandraTestContext)
+    throws Exception
+    {
+        BBHelperMultipleJoiningNodes.reset();
+        runJoiningTestScenario(context,
+                               cassandraTestContext,
+                               BBHelperMultipleJoiningNodes::install,
+                               BBHelperMultipleJoiningNodes.transientStateStart,
+                               BBHelperMultipleJoiningNodes.transientStateEnd,
+                               generateExpectedRangeMappingMultipleJoiningNodes());
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 3 node cluster
+     * with the 2 more nodes joining the cluster
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * We generate the expected ranges by using
+     * 1) the initial token allocations to nodes (prior to adding nodes) shown under "Initial Ranges"
+     * (in the comment block below),
+     * 2)the "pending node ranges" and
+     * 3) the final token allocations per node.
+     * <p>
+     * Step 1: Prepare ranges starting from partitioner min-token, ending at partitioner max-token using (3) above
+     * Step 2: Create the cascading list of replica-sets based on the RF (3) for each range using the initial node list
+     * Step 3: Add replicas to ranges based on (1) and (2) above
+     */
+    private HashMap<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingMultipleJoiningNodes()
+    {
+        /*
+         * All ranges previously had replicas 1, 2, 3, since this was a 3 node cluster with RF = 3
+         *
+         * Initial Ranges:
+         * [-9223372036854775808, -4611686018427387905]:["127.0.0.3","127.0.0.2","127.0.0.1"]
+         * [-4611686018427387905, -3]:["127.0.0.3","127.0.0.2","127.0.0.1"]
+         * [-3, 4611686018427387899]:["127.0.0.3","127.0.0.2","127.0.0.1"]
+         * [4611686018427387899, 9223372036854775807]:["127.0.0.3","127.0.0.2","127.0.0.1"]
+         *
+         * Pending ranges:
+         * [-3, 4611686018427387899]=[127.0.0.4:62469]
+         * [-4611686018427387905, -3]=[127.0.0.5:62472]
+         * [-4611686018427387905, -2305843009213693954]=[127.0.0.4:62469]
+         * [4611686018427387899, -4611686018427387905]=[127.0.0.4:62469, 127.0.0.5:62472] (wrap-around)
+         * [-3, 2305843009213693948]=[127.0.0.5:62472]
+         *
+         * Token assignment for new nodes:
+         * 127.0.0.4 - [-2305843009213693954]
+         * 127.0.0.5 - [2305843009213693948]
+         *
+         * Based on the pending ranges, we add the expected replicas to the ranges they intersect below
+         */
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        // [-9223372036854775808, -4611686018427387905]
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4",
+                                                         "127.0.0.5"));
+        // [-4611686018427387905, -2305843009213693954]
+        mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.1", "127.0.0.4",
+                                                         "127.0.0.5"));
+        // [-2305843009213693954, -3]
+        mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.1", "127.0.0.2", "127.0.0.5"));
+        // [-3, 2305843009213693948]
+        mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4",
+                                                         "127.0.0.5"));
+        // [2305843009213693948, 4611686018427387899]
+        mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.4", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        // [4611686018427387899, 9223372036854775807]
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4",
+                                                         "127.0.0.5"));
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for multiple joining nodes
+     */
+    @Shared
+    public static class BBHelperMultipleJoiningNodes
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(2);
+        static CountDownLatch transientStateEnd = 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
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(2);
+            transientStateEnd = new CountDownLatch(2);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestSingleNode.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestSingleNode.java
new file mode 100644
index 0000000..6b285d8
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/JoiningTestSingleNode.java
@@ -0,0 +1,156 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Cluster expansion scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+public class JoiningTestSingleNode extends JoiningBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 1, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithJoiningNode(VertxTestContext context,
+                                        ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperSingleJoiningNode.reset();
+        runJoiningTestScenario(context,
+                               cassandraTestContext,
+                               BBHelperSingleJoiningNode::install,
+                               BBHelperSingleJoiningNode.transientStateStart,
+                               BBHelperSingleJoiningNode.transientStateEnd,
+                               generateExpectedRangeMappingSingleJoiningNode());
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * with the additional node joining the cluster
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * Ranges that include the joining node will have [RF + no. joining nodes in replica-set] replicas with
+     * the replicas being the existing nodes in ring-order.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with E being the joining node)
+     * Expected Range 2 - B, C, D, E
+     */
+    private HashMap<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingSingleJoiningNode()
+    {
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.6"));
+        mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.6"));
+
+        mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1"));
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2", "127.0.0.6"));
+        mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.6"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for a single joining node
+     */
+    @Shared
+    public static class BBHelperSingleJoiningNode
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(1);
+        static CountDownLatch transientStateEnd = 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
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(1);
+            transientStateEnd = new CountDownLatch(1);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingBaseTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingBaseTest.java
new file mode 100644
index 0000000..f91e493
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingBaseTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+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 io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.ClusterUtils;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Base class for TokenRangeIntegrationLeaving Tests
+ */
+class LeavingBaseTest extends BaseTokenRangeIntegrationTest
+{
+    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)
+            {
+                createTestKeyspace(ImmutableMap.of("replication_factor", DEFAULT_RF));
+                dcReplication = Sets.newHashSet(Arrays.asList("datacenter1", "datacenter2"));
+            }
+            else
+            {
+                createTestKeyspace(ImmutableMap.of("datacenter1", DEFAULT_RF));
+                dcReplication = Collections.singleton("datacenter1");
+            }
+
+            IUpgradeableInstance seed = cluster.get(1);
+
+            List<IUpgradeableInstance> leavingNodes = new ArrayList<>();
+            for (int i = 0; i < leavingNodesPerDC * annotation.numDcs(); i++)
+            {
+                IUpgradeableInstance node = cluster.get(cluster.size() - i);
+                new Thread(() -> node.nodetoolResult("decommission").asserts().success()).start();
+                leavingNodes.add(node);
+            }
+
+            // Wait until nodes have reached expected state
+            Uninterruptibles.awaitUninterruptibly(transientStateStart);
+
+            for (IUpgradeableInstance node : leavingNodes)
+            {
+                ClusterUtils.awaitRingState(seed, node, "Leaving");
+            }
+
+            retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+                assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
+                TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+                assertMappingResponseOK(mappingResponse,
+                                        DEFAULT_RF,
+                                        dcReplication);
+
+                int initialNodeCount = annotation.nodesPerDc() * annotation.numDcs();
+                validateNodeStates(mappingResponse,
+                                   dcReplication,
+                                   nodeNumber ->
+                                   nodeNumber <= (initialNodeCount - (leavingNodesPerDC * annotation.numDcs())) ?
+                                   "Normal" :
+                                   "Leaving");
+                validateTokenRanges(mappingResponse, generateExpectedRanges());
+                validateReplicaMapping(mappingResponse, leavingNodes, expectedRangeMappings);
+
+                context.completeNow();
+            });
+        }
+        finally
+        {
+            for (int i = 0; i < leavingNodesPerDC; i++)
+            {
+                transientStateEnd.countDown();
+            }
+        }
+    }
+
+    private void validateReplicaMapping(TokenRangeReplicasResponse mappingResponse,
+                                        List<IUpgradeableInstance> leavingNodes,
+                                        Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    {
+        List<String> transientNodeAddresses = leavingNodes.stream().map(i -> {
+            InetSocketAddress address = i.config().broadcastAddress();
+            return address.getAddress().getHostAddress() +
+                   ":" +
+                   address.getPort();
+        }).collect(Collectors.toList());
+
+        Set<String> writeReplicaInstances = instancesFromReplicaSet(mappingResponse.writeReplicas());
+        Set<String> readReplicaInstances = instancesFromReplicaSet(mappingResponse.readReplicas());
+        assertThat(readReplicaInstances).containsAll(transientNodeAddresses);
+        assertThat(writeReplicaInstances).containsAll(transientNodeAddresses);
+
+        validateWriteReplicaMappings(mappingResponse.writeReplicas(), expectedRangeMappings);
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTest.java
new file mode 100644
index 0000000..b114564
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTest.java
@@ -0,0 +1,385 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.function.BiConsumer;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.TokenSupplier;
+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;
+
+/**
+ * Cluster shrink scenarios integration tests for token range replica mapping endpoint with the in-jvm dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+class LeavingTest extends LeavingBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithKeyspaceLeavingNode(VertxTestContext context,
+                                                ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperSingleLeavingNode.reset();
+        runLeavingTestScenario(context,
+                               cassandraTestContext,
+                               1,
+                               BBHelperSingleLeavingNode::install,
+                               BBHelperSingleLeavingNode.transientStateStart,
+                               BBHelperSingleLeavingNode.transientStateEnd,
+                               generateExpectedRangeMappingSingleLeavingNode());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithMultipleLeavingNodes(VertxTestContext context,
+                                                 ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperMultipleLeavingNodes.reset();
+        runLeavingTestScenario(context,
+                               cassandraTestContext,
+                               2,
+                               BBHelperMultipleLeavingNodes::install,
+                               BBHelperMultipleLeavingNodes.transientStateStart,
+                               BBHelperMultipleLeavingNodes.transientStateEnd,
+                               generateExpectedRangeMappingMultipleLeavingNodes());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 6, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingHalveClusterSize(VertxTestContext context,
+                                         ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperHalveClusterSize.reset();
+        runLeavingTestScenario(context,
+                               cassandraTestContext,
+                               3,
+                               BBHelperHalveClusterSize::install,
+                               BBHelperHalveClusterSize.transientStateStart,
+                               BBHelperHalveClusterSize.transientStateEnd,
+                               generateExpectedRangeMappingHalveClusterSize());
+    }
+
+    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
+    {
+
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                annotation.newNodesPerDc(),
+                                                                                annotation.numDcs(),
+                                                                                1);
+
+        UpgradeableCluster cluster = cassandraTestContext.configureAndStartCluster(builder -> {
+            builder.withInstanceInitializer(instanceInitializer);
+            builder.withTokenSupplier(tokenSupplier);
+        });
+        runLeavingTestScenario(context,
+                               leavingNodesPerDC,
+                               transientStateStart,
+                               transientStateEnd,
+                               cluster,
+                               expectedRangeMappings);
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * with the last node leaving the cluster
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * Ranges that including leaving node replicas will have [RF + no. leaving nodes in replica-set] replicas with
+     * the new replicas being the existing nodes in ring-order.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being the leaving node)
+     * Expected Range 2 - B, C, D, A (With A taking over the range of the leaving node)
+     */
+    private HashMap<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingSingleLeavingNode()
+    {
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+
+        mapping.put(expectedRanges.get(2),
+                    Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.1"));
+        mapping.put(expectedRanges.get(3),
+                    Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1", "127.0.0.2"));
+        mapping.put(expectedRanges.get(4),
+                    Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * with the last 2 nodes leaving the cluster
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * Ranges that including leaving node replicas will have [RF + no. leaving nodes in replica-set] replicas with
+     * the new replicas being the existing nodes in ring-order.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being the leaving node)
+     * Expected Range 2 - B, C, D, A (With A taking over the range of the leaving node)
+     */
+
+    private HashMap<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingMultipleLeavingNodes()
+    {
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(expectedRanges.get(1),
+                    Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.1"));
+        mapping.put(
+        expectedRanges.get(2),
+        Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.2", "127.0.0.1"));
+        mapping.put(
+        expectedRanges.get(3),
+        Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(
+        expectedRanges.get(4),
+        Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 6 node cluster
+     * with the last 3 nodes leaving the cluster
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * Ranges that including leaving node replicas will have [RF + no. leaving nodes in replica-set] replicas with
+     * the new replicas being the existing nodes in ring-order.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being the leaving node)
+     * Expected Range 2 - B, C, D, A (With A taking over the range of the leaving node)
+     */
+
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingHalveClusterSize()
+    {
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(
+        expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.1"));
+        mapping.put(
+        expectedRanges.get(2),
+        Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.1", "127.0.0.2"));
+        mapping.put(
+        expectedRanges.get(3),
+        Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.1", "127.0.0.2",
+                      "127.0.0.3"));
+        mapping.put(
+        expectedRanges.get(4),
+        Arrays.asList("127.0.0.5", "127.0.0.6", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(
+        expectedRanges.get(5), Arrays.asList("127.0.0.6", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy Helper for a single leaving node
+     */
+    @Shared
+    public static class BBHelperSingleLeavingNode
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(1);
+        static CountDownLatch transientStateEnd = 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
+        {
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            orig.call();
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(1);
+            transientStateEnd = new CountDownLatch(1);
+        }
+    }
+
+    /**
+     * ByteBuddy helper for multiple leaving nodes
+     */
+    @Shared
+    public static class BBHelperMultipleLeavingNodes
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(2);
+        static CountDownLatch transientStateEnd = new CountDownLatch(2);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 5 node cluster with a 2 leaving nodes
+            // We intercept the shutdown of the leaving 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("unbootstrap"))
+                               .intercept(MethodDelegation.to(BBHelperMultipleLeavingNodes.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
+        {
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            orig.call();
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(2);
+            transientStateEnd = new CountDownLatch(2);
+        }
+    }
+
+    /**
+     * ByteBuddy helper for shrinking cluster by half its size
+     */
+    @Shared
+    public static class BBHelperHalveClusterSize
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(3);
+        static CountDownLatch transientStateEnd = new CountDownLatch(3);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves halving the size of a 6 node cluster
+            // We intercept the shutdown of the removed nodes (4-6) 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("unbootstrap"))
+                               .intercept(MethodDelegation.to(BBHelperHalveClusterSize.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
+        {
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            orig.call();
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(3);
+            transientStateEnd = new CountDownLatch(3);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTestMultiDC.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTestMultiDC.java
new file mode 100644
index 0000000..77173df
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTestMultiDC.java
@@ -0,0 +1,197 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import org.apache.cassandra.utils.Shared;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+
+/**
+ * Multi-DC Cluster shrink scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+class LeavingTestMultiDC extends LeavingBaseTest
+{
+    @CassandraIntegrationTest(
+    nodesPerDc = 5, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithLeavingNodesMultiDC(VertxTestContext context,
+                                                ConfigurableCassandraTestContext cassandraTestContext)
+    throws Exception
+    {
+        BBHelperLeavingNodesMultiDC.reset();
+        int leavingNodesPerDC = 1;
+        UpgradeableCluster cluster = getMultiDCCluster(BBHelperLeavingNodesMultiDC::install, cassandraTestContext);
+
+        runLeavingTestScenario(context,
+                               leavingNodesPerDC,
+                               BBHelperLeavingNodesMultiDC.transientStateStart,
+                               BBHelperLeavingNodesMultiDC.transientStateEnd,
+                               cluster,
+                               generateExpectedRangeMappingLeavingNodeMultiDC());
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 10 node cluster
+     * across 2 DCs with the last 2 nodes leaving the cluster (1 per DC), with RF 3
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * In a multi-DC scenario, a single range will have nodes from both DCs. The replicas are grouped by DC here
+     * to allow per-DC validation as returned from the sidecar endpoint.
+     * <p>
+     * Ranges that including leaving node replicas will have [RF + no. leaving nodes in replica-set] replicas with
+     * the new replicas being the existing nodes in ring-order.
+     * <p>
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being the leaving node)
+     * Expected Range 2 - B, C, D, A (With A taking over the range of the leaving node)
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingLeavingNodeMultiDC()
+    {
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> dc1Mapping = new HashMap<>();
+        Map<Range<BigInteger>, List<String>> dc2Mapping = new HashMap<>();
+
+        dc1Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7"));
+        dc2Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7"));
+        dc2Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8"));
+
+        dc1Mapping.put(expectedRanges.get(3),
+                       Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8"));
+
+        dc1Mapping.put(expectedRanges.get(4),
+                       Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(4),
+                       Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10", "127.0.0.2"));
+
+        dc1Mapping.put(expectedRanges.get(5),
+                       Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1", "127.0.0.3"));
+        dc2Mapping.put(expectedRanges.get(5),
+                       Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10", "127.0.0.2"));
+
+        dc1Mapping.put(expectedRanges.get(6),
+                       Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1", "127.0.0.3"));
+        dc2Mapping.put(expectedRanges.get(6),
+                       Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.2", "127.0.0.4"));
+
+        dc1Mapping.put(expectedRanges.get(7),
+                       Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(7),
+                       Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.2", "127.0.0.4"));
+
+        dc1Mapping.put(expectedRanges.get(8),
+                       Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(8),
+                       Arrays.asList("127.0.0.10", "127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(9),
+                       Arrays.asList("127.0.0.10", "127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", dc1Mapping);
+                put("datacenter2", dc2Mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for multiple leaving nodes multi-DC
+     */
+    @Shared
+    public static class BBHelperLeavingNodesMultiDC
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(2);
+        static CountDownLatch transientStateEnd = new CountDownLatch(2);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 10 node cluster (5 nodes per DC) with a 2 leaving nodes (1 per DC)
+            // We intercept the shutdown of the leaving nodes (9, 10) to validate token ranges
+            if (nodeNumber > 8)
+            {
+                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(BBHelperLeavingNodesMultiDC.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
+        {
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            orig.call();
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(2);
+            transientStateEnd = new CountDownLatch(2);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTestMultiDCHalveCluster.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTestMultiDCHalveCluster.java
new file mode 100644
index 0000000..6811116
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/LeavingTestMultiDCHalveCluster.java
@@ -0,0 +1,230 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import org.apache.cassandra.utils.Shared;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+
+/**
+ * Multi-DC Cluster shrink scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ *
+ * Note: Some related test classes are broken down to have a single test case to parallelize test execution and
+ * therefore limit the instance size required to run the tests from CircleCI as the in-jvm-dtests tests are memory bound
+ */
+@ExtendWith(VertxExtension.class)
+class LeavingTestMultiDCHalveCluster extends LeavingBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 6, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingMultiDCHalveClusterSize(VertxTestContext context,
+                                                ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperHalveClusterMultiDC.reset();
+        int leavingNodesPerDC = 3;
+        UpgradeableCluster cluster = getMultiDCCluster(BBHelperHalveClusterMultiDC::install, cassandraTestContext);
+
+        Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings
+        = generateExpectedRangeHalveClusterSizeMultiDC();
+        runLeavingTestScenario(context,
+                               leavingNodesPerDC,
+                               BBHelperHalveClusterMultiDC.transientStateStart,
+                               BBHelperHalveClusterMultiDC.transientStateEnd,
+                               cluster,
+                               expectedRangeMappings);
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 12 node cluster
+     * across 2 DCs with the last 6 nodes leaving the cluster (3 per DC)
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * In a multi-DC scenario, a single range will have nodes from both DCs. The replicas are grouped by DC here
+     * to allow per-DC validation as returned from the sidecar endpoint.
+     * <p>
+     * Ranges that including leaving node replicas will have [RF + no. leaving nodes in replica-set] replicas with
+     * the new replicas being the existing nodes in ring-order.
+     * <p>
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being the leaving node)
+     * Expected Range 2 - B, C, D, A (With A taking over the range of the leaving node)
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeHalveClusterSizeMultiDC()
+    {
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges();
+        Map<Range<BigInteger>, List<String>> dc1Mapping = new HashMap<>();
+        Map<Range<BigInteger>, List<String>> dc2Mapping = new HashMap<>();
+
+        dc1Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(1),
+                       Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(2),
+                       Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(2),
+                       Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8", "127.0.0.2"));
+
+        dc1Mapping.put(
+        expectedRanges.get(3),
+        Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9", "127.0.0.1", "127.0.0.3"));
+        dc2Mapping.put(expectedRanges.get(3),
+                       Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8", "127.0.0.2"));
+
+        dc1Mapping.put(
+        expectedRanges.get(4),
+        Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9", "127.0.0.1", "127.0.0.3"));
+        dc2Mapping.put(
+        expectedRanges.get(4),
+        Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10", "127.0.0.2", "127.0.0.4"));
+
+        dc1Mapping.put(
+        expectedRanges.get(5),
+        Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.11", "127.0.0.1", "127.0.0.3",
+                      "127.0.0.5"));
+        dc2Mapping.put(
+        expectedRanges.get(5),
+        Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10", "127.0.0.2", "127.0.0.4"));
+
+        dc1Mapping.put(
+        expectedRanges.get(6),
+        Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.11", "127.0.0.1", "127.0.0.3",
+                      "127.0.0.5"));
+        dc2Mapping.put(
+        expectedRanges.get(6),
+        Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.12", "127.0.0.2", "127.0.0.4",
+                      "127.0.0.6"));
+
+        dc1Mapping.put(
+        expectedRanges.get(7),
+        Arrays.asList("127.0.0.9", "127.0.0.11", "127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(
+        expectedRanges.get(7),
+        Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.12", "127.0.0.2", "127.0.0.4",
+                      "127.0.0.6"));
+
+        dc1Mapping.put(
+        expectedRanges.get(8),
+        Arrays.asList("127.0.0.9", "127.0.0.11", "127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(
+        expectedRanges.get(8),
+        Arrays.asList("127.0.0.10", "127.0.0.12", "127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(9),
+                       Arrays.asList("127.0.0.11", "127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(
+        expectedRanges.get(9),
+        Arrays.asList("127.0.0.10", "127.0.0.12", "127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(10),
+                       Arrays.asList("127.0.0.11", "127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(10),
+                       Arrays.asList("127.0.0.12", "127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(11),
+                       Arrays.asList("127.0.0.12", "127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(12), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(12), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", dc1Mapping);
+                put("datacenter2", dc2Mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for halve cluster size with multi-DC
+     */
+    @Shared
+    public static class BBHelperHalveClusterMultiDC
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(6);
+        static CountDownLatch transientStateEnd = new CountDownLatch(6);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves halving the size of a 12 node cluster (6 per DC)
+            // We intercept the shutdown of the removed nodes (7-12) 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("unbootstrap"))
+                               .intercept(MethodDelegation.to(BBHelperHalveClusterMultiDC.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
+        {
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            orig.call();
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(6);
+            transientStateEnd = new CountDownLatch(6);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingBaseTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingBaseTest.java
new file mode 100644
index 0000000..22d3b0e
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingBaseTest.java
@@ -0,0 +1,202 @@
+/*
+ * 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.tokenrange;
+
+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.Map;
+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 io.netty.handler.codec.http.HttpResponseStatus;
+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 static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Base class for TokenRangeIntegrationMoving Tests
+ */
+class MovingBaseTest extends BaseTokenRangeIntegrationTest
+{
+    public static final int MOVING_NODE_IDX = 5;
+    public static final int MULTIDC_MOVING_NODE_IDX = 10;
+
+    void runMovingTestScenario(VertxTestContext context,
+                               CountDownLatch transientStateStart,
+                               CountDownLatch transientStateEnd,
+                               UpgradeableCluster cluster,
+                               Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings,
+                               long moveTargetToken) throws Exception
+    {
+        try
+        {
+            CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+            Set<String> dcReplication;
+            if (annotation.numDcs() > 1)
+            {
+                createTestKeyspace(ImmutableMap.of("replication_factor", DEFAULT_RF));
+                dcReplication = Sets.newHashSet(Arrays.asList("datacenter1", "datacenter2"));
+            }
+            else
+            {
+                createTestKeyspace(ImmutableMap.of("datacenter1", DEFAULT_RF));
+                dcReplication = Collections.singleton("datacenter1");
+            }
+
+            IUpgradeableInstance seed = cluster.get(1);
+            int movingNodeIndex = (annotation.numDcs() > 1) ? MULTIDC_MOVING_NODE_IDX : MOVING_NODE_IDX;
+
+            IUpgradeableInstance movingNode = cluster.get(movingNodeIndex);
+            new Thread(() -> movingNode.nodetoolResult("move", "--", Long.toString(moveTargetToken))
+                                       .asserts()
+                                       .success()).start();
+
+            // Wait until nodes have reached expected state
+            Uninterruptibles.awaitUninterruptibly(transientStateStart, 2, TimeUnit.MINUTES);
+            ClusterUtils.awaitRingState(seed, movingNode, "Moving");
+
+            retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+                assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
+                TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+                assertMappingResponseOK(mappingResponse,
+                                        DEFAULT_RF,
+                                        dcReplication);
+
+                validateNodeStates(mappingResponse,
+                                   dcReplication,
+                                   nodeNumber -> nodeNumber == movingNodeIndex ? "Moving" : "Normal");
+                List<Range<BigInteger>> expectedRanges = getMovingNodesExpectedRanges(annotation.nodesPerDc(),
+                                                                                      annotation.numDcs(),
+                                                                                      moveTargetToken);
+                validateTokenRanges(mappingResponse, expectedRanges);
+                validateReplicaMapping(mappingResponse, movingNode, moveTargetToken, expectedRangeMappings);
+
+                context.completeNow();
+            });
+        }
+        finally
+        {
+            transientStateEnd.countDown();
+        }
+    }
+
+
+    private void validateReplicaMapping(TokenRangeReplicasResponse mappingResponse,
+                                        IUpgradeableInstance movingNode,
+                                        long moveTo,
+                                        Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    {
+        InetSocketAddress address = movingNode.config().broadcastAddress();
+        String expectedAddress = address.getAddress().getHostAddress() +
+                                 ":" +
+                                 address.getPort();
+
+        Set<String> writeReplicaInstances = instancesFromReplicaSet(mappingResponse.writeReplicas());
+        Set<String> readReplicaInstances = instancesFromReplicaSet(mappingResponse.readReplicas());
+
+        Optional<TokenRangeReplicasResponse.ReplicaInfo> moveResultRange // Get ranges ending in move token
+        = mappingResponse.writeReplicas()
+                         .stream()
+                         .filter(r -> r.end().equals(String.valueOf(moveTo)))
+                         .findAny();
+        assertThat(moveResultRange.isPresent());
+        List<String> replicasInRange = moveResultRange.get().replicasByDatacenter().values()
+                                                      .stream()
+                                                      .flatMap(Collection::stream)
+                                                      .collect(Collectors.toList());
+        assertThat(replicasInRange).contains(expectedAddress);
+        assertThat(readReplicaInstances).contains(expectedAddress);
+        assertThat(writeReplicaInstances).contains(expectedAddress);
+
+        validateWriteReplicaMappings(mappingResponse.writeReplicas(), expectedRangeMappings);
+    }
+
+    protected List<Range<BigInteger>> getMovingNodesExpectedRanges(int initialNodeCount, int numDcs, long moveTo)
+    {
+        boolean moveHandled = false;
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                annotation.newNodesPerDc(),
+                                                                                annotation.numDcs(),
+                                                                                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 <= (initialNodeCount * numDcs))
+        {
+
+            BigInteger currentToken = new BigInteger(tokenSupplier.tokens(node).stream().findFirst().get());
+            if (!moveHandled && currentToken.compareTo(BigInteger.valueOf(moveTo)) > 0)
+            {
+                expectedRanges.add(Range.openClosed(prevToken, BigInteger.valueOf(moveTo)));
+                expectedRanges.add(Range.openClosed(BigInteger.valueOf(moveTo), currentToken));
+                moveHandled = true;
+            }
+            else
+            {
+                expectedRanges.add(Range.openClosed(prevToken, currentToken));
+            }
+
+            prevToken = currentToken;
+            node++;
+        }
+        expectedRanges.add(Range.openClosed(prevToken, endToken));
+
+        return expectedRanges;
+    }
+
+    protected long getMoveTargetToken(UpgradeableCluster cluster)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        IUpgradeableInstance seed = cluster.get(1);
+        // The target token to move the node to is calculated by adding an offset to the seed node token which
+        // is half of the range between 2 tokens.
+        // For multi-DC case (specifically 2 DCs), since neighbouring tokens can be consecutive, we use tokens 1
+        // and 3 to calculate the offset
+        int nextIndex = (annotation.numDcs() > 1) ? 3 : 2;
+        long t2 = Long.parseLong(seed.config().getString("initial_token"));
+        long t3 = Long.parseLong(cluster.get(nextIndex).config().getString("initial_token"));
+        return (t2 + ((t3 - t2) / 2));
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingMultiDCTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingMultiDCTest.java
new file mode 100644
index 0000000..a486de6
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingMultiDCTest.java
@@ -0,0 +1,214 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import org.apache.cassandra.utils.Shared;
+
+import static net.bytebuddy.matcher.ElementMatchers.named;
+
+/**
+ * Multi-DC Node movement scenarios integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+class MovingMultiDCTest extends MovingBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWhileMovingNodeMultiDC(VertxTestContext context,
+                                               ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperMovingNodeMultiDC.reset();
+        UpgradeableCluster cluster = getMultiDCCluster(BBHelperMovingNodeMultiDC::install, cassandraTestContext);
+
+        long moveTarget = getMoveTargetToken(cluster);
+        runMovingTestScenario(context,
+                              BBHelperMovingNodeMultiDC.transientStateStart,
+                              BBHelperMovingNodeMultiDC.transientStateEnd,
+                              cluster,
+                              generateExpectedRangeMappingMovingNodeMultiDC(moveTarget),
+                              moveTarget);
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 10 node cluster
+     * across 2 DCs with the last node being moved by assigning it a different token
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * In this test case, the moved node is inserted between nodes 1 and 2, resulting in splitting the ranges.
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>>
+    generateExpectedRangeMappingMovingNodeMultiDC(long moveTarget)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        List<Range<BigInteger>> expectedRanges = getMovingNodesExpectedRanges(annotation.nodesPerDc(),
+                                                                              annotation.numDcs(),
+                                                                              moveTarget);
+        /*
+         * The following expected ranges are generated based on the following token assignments and pending ranges.
+         *
+         * Token Allocations:
+         * MIN TOKEN: -9223372036854775808
+         * /127.0.0.1:[-5534023222112865487]
+         * /127.0.0.2:[-5534023222112865486]
+         * /127.0.0.3:[-1844674407370955167]
+         * /127.0.0.10:[-3689348814741910327] (New Location)
+         * /127.0.0.4:[-1844674407370955166]
+         * /127.0.0.5:[1844674407370955153]
+         * /127.0.0.6:[1844674407370955154]
+         * /127.0.0.7:[5534023222112865473]
+         * /127.0.0.8:[5534023222112865474]
+         * /127.0.0.9:[9223372036854775793]
+         * /127.0.0.10:[9223372036854775793] (Old Location)
+         * MAX TOKEN: 9223372036854775807
+         *
+         * Pending Ranges:
+         * [-5534023222112865487, -5534023222112865486]=[127.0.0.10]
+         * [-5534023222112865486, -3689348814741910327]=[127.0.0.10]
+         * [-1844674407370955166, 1844674407370955153]=[127.0.0.2]
+         * [1844674407370955153, 1844674407370955154]=[127.0.0.2]
+         * [9223372036854775794, -5534023222112865487]=[127.0.0.10]
+         */
+
+        Map<Range<BigInteger>, List<String>> dc1Mapping = new HashMap<>();
+        Map<Range<BigInteger>, List<String>> dc2Mapping = new HashMap<>();
+        // Replica 2
+        dc1Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6",
+                                                            "127.0.0.10"));
+
+        dc1Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7"));
+        dc2Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6",
+                                                            "127.0.0.10"));
+        // Split range resulting from the new token. Part 1 including the new token.
+        dc1Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7"));
+        dc2Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8",
+                                                            "127.0.0.10"));
+        // Split range resulting from the new token. Part 2 excluding new token (but starting from it)
+        dc1Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7"));
+        dc2Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8"));
+
+        dc1Mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8"));
+
+        dc1Mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10",
+                                                            "127.0.0.2"));
+
+        dc1Mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10",
+                                                            "127.0.0.2"));
+
+        dc1Mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.2"));
+
+        dc1Mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3"));
+        dc2Mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.2"));
+
+        dc1Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3"));
+        dc2Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.10", "127.0.0.2", "127.0.0.4"));
+
+        dc1Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.10", "127.0.0.2", "127.0.0.4"));
+        // Replica 3
+        dc1Mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5"));
+        dc2Mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6",
+                                                             "127.0.0.10"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", dc1Mapping);
+                put("datacenter2", dc2Mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy Helper for a multiDC moving node
+     */
+    @Shared
+    public static class BBHelperMovingNodeMultiDC
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(1);
+        static CountDownLatch transientStateEnd = new CountDownLatch(1);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Moving the 5th node in the test case
+            if (nodeNumber == MULTIDC_MOVING_NODE_IDX)
+            {
+                TypePool typePool = TypePool.Default.of(cl);
+                TypeDescription description = typePool.describe("org.apache.cassandra.service.RangeRelocator")
+                                                      .resolve();
+                new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
+                               .method(named("stream"))
+                               .intercept(MethodDelegation.to(BBHelperMovingNodeMultiDC.class))
+                               // Defer class loading until all dependencies are loaded
+                               .make(TypeResolutionStrategy.Lazy.INSTANCE, typePool)
+                               .load(cl, ClassLoadingStrategy.Default.INJECTION);
+            }
+        }
+
+        @SuppressWarnings("unused")
+        public static Future<?> stream(@SuperCall Callable<Future<?>> orig) throws Exception
+        {
+            Future<?> res = orig.call();
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return res;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(1);
+            transientStateEnd = new CountDownLatch(1);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingTest.java
new file mode 100644
index 0000000..5b0238a
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/MovingTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.TokenSupplier;
+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;
+
+/**
+ * Node movement scenarios integration tests for token range replica mapping endpoint with the in-jvm dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+class MovingTest extends MovingBaseTest
+{
+
+    @CassandraIntegrationTest(nodesPerDc = 5, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithKeyspaceMovingNode(VertxTestContext context,
+                                               ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperMovingNode.reset();
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                annotation.newNodesPerDc(),
+                                                                                annotation.numDcs(),
+                                                                                1);
+
+        UpgradeableCluster cluster = cassandraTestContext.configureAndStartCluster(builder -> {
+            builder.withInstanceInitializer(BBHelperMovingNode::install);
+            builder.withTokenSupplier(tokenSupplier);
+        });
+
+        long moveTarget = getMoveTargetToken(cluster);
+        runMovingTestScenario(context,
+                              BBHelperMovingNode.transientStateStart,
+                              BBHelperMovingNode.transientStateEnd,
+                              cluster,
+                              generateExpectedRangeMappingMovingNode(moveTarget),
+                              moveTarget);
+    }
+
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * with the last node being moved by assigning it a different token
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in subsequent
+     * ranges cascade with the next range excluding the first replica, and including the next replica from the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * In this test case, the moved node is inserted between nodes 1 and 2, resulting in splitting the ranges.
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingMovingNode(long moveTarget)
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        List<Range<BigInteger>> expectedRanges = getMovingNodesExpectedRanges(annotation.nodesPerDc(),
+                                                                              annotation.numDcs(),
+                                                                              moveTarget);
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        // Initial range from Partitioner's MIN_TOKEN. This will include one of the replicas of the moved node since
+        // it is adjacent to the range where it is being introduced.
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3",
+                                                         "127.0.0.5"));
+        // Range including the token of the moved node. Node 5 is added here (and the preceding 3 ranges)
+        mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4",
+                                                         "127.0.0.5"));
+        // Split range resulting from the new token. This range is exclusive of the new token and node 5, and
+        // has the same replicas as the previous range (as a result of the split)
+        mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        // Node 1 is introduced here as it will take ownership of a portion of node 5's previous tokens as a result
+        // of the move.
+        mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5",
+                                                         "127.0.0.1"));
+        // Following 2 ranges remain unchanged as the replica-set remain the same post-move
+        mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1"));
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2"));
+        // Third (wrap-around) replica of the new location of node 5 is added to the existing replica-set
+        mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3",
+                                                         "127.0.0.5"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy Helper for a single moving node
+     */
+    @Shared
+    public static class BBHelperMovingNode
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(1);
+        static CountDownLatch transientStateEnd = new CountDownLatch(1);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Moving the 5th node in the test case
+            if (nodeNumber == MOVING_NODE_IDX)
+            {
+                TypePool typePool = TypePool.Default.of(cl);
+                TypeDescription description = typePool.describe("org.apache.cassandra.service.RangeRelocator")
+                                                      .resolve();
+                new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
+                               .method(named("stream"))
+                               .intercept(MethodDelegation.to(BBHelperMovingNode.class))
+                               // Defer class loading until all dependencies are loaded
+                               .make(TypeResolutionStrategy.Lazy.INSTANCE, typePool)
+                               .load(cl, ClassLoadingStrategy.Default.INJECTION);
+            }
+        }
+
+        @SuppressWarnings("unused")
+        public static Future<?> stream(@SuperCall Callable<Future<?>> orig) throws Exception
+        {
+            Future<?> res = orig.call();
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return res;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(1);
+            transientStateEnd = new CountDownLatch(1);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementBaseTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementBaseTest.java
new file mode 100644
index 0000000..a5b3576
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementBaseTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+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 io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.shared.ClusterUtils;
+import org.apache.cassandra.sidecar.common.data.TokenRangeReplicasResponse;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Base class for the TokenRangeIntegrationReplacement Tests
+ */
+class ReplacementBaseTest extends BaseTokenRangeIntegrationTest
+{
+    protected void runReplacementTestScenario(VertxTestContext context,
+                                              CountDownLatch nodeStart,
+                                              CountDownLatch transientStateStart,
+                                              CountDownLatch transientStateEnd,
+                                              UpgradeableCluster cluster,
+                                              List<IUpgradeableInstance> nodesToRemove,
+                                              Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    throws Exception
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        try
+        {
+            Set<String> dcReplication;
+            if (annotation.numDcs() > 1)
+            {
+                createTestKeyspace(ImmutableMap.of("replication_factor", DEFAULT_RF));
+                dcReplication = Sets.newHashSet(Arrays.asList("datacenter1", "datacenter2"));
+            }
+            else
+            {
+                createTestKeyspace(ImmutableMap.of("datacenter1", DEFAULT_RF));
+                dcReplication = Collections.singleton("datacenter1");
+            }
+
+            IUpgradeableInstance seed = cluster.get(1);
+            List<String> removedNodeAddresses = nodesToRemove.stream()
+                                                             .map(n ->
+                                                                  n.config()
+                                                                   .broadcastAddress()
+                                                                   .getAddress()
+                                                                   .getHostAddress())
+                                                             .collect(Collectors.toList());
+
+            List<ClusterUtils.RingInstanceDetails> ring = ClusterUtils.ring(seed);
+            List<String> removedNodeTokens = ring.stream()
+                                                 .filter(i -> removedNodeAddresses.contains(i.getAddress()))
+                                                 .map(ClusterUtils.RingInstanceDetails::getToken)
+                                                 .collect(Collectors.toList());
+
+            stopNodes(seed, nodesToRemove);
+            List<IUpgradeableInstance> newNodes = startReplacementNodes(nodeStart, cluster, nodesToRemove);
+
+            // Wait until replacement nodes are in JOINING state
+            Uninterruptibles.awaitUninterruptibly(transientStateStart, 2, TimeUnit.MINUTES);
+
+            // Verify state of replacement nodes
+            for (IUpgradeableInstance newInstance : newNodes)
+            {
+                ClusterUtils.awaitRingState(newInstance, newInstance, "Joining");
+                ClusterUtils.awaitGossipStatus(newInstance, newInstance, "BOOT_REPLACE");
+
+                String newAddress = newInstance.config().broadcastAddress().getAddress().getHostAddress();
+                Optional<ClusterUtils.RingInstanceDetails> replacementInstance = ClusterUtils.ring(seed)
+                                                                                             .stream()
+                                                                                             .filter(
+                                                                                             i -> i.getAddress()
+                                                                                                   .equals(newAddress))
+                                                                                             .findFirst();
+                assertThat(replacementInstance).isPresent();
+                // Verify that replacement node tokens match the removed nodes
+                assertThat(removedNodeTokens).contains(replacementInstance.get().getToken());
+            }
+
+            retrieveMappingWithKeyspace(context, TEST_KEYSPACE, response -> {
+                assertThat(response.statusCode()).isEqualTo(HttpResponseStatus.OK.code());
+                TokenRangeReplicasResponse mappingResponse = response.bodyAsJson(TokenRangeReplicasResponse.class);
+                assertMappingResponseOK(mappingResponse,
+                                        DEFAULT_RF,
+                                        dcReplication);
+
+                List<Integer> nodeNums = newNodes.stream().map(i -> i.config().num()).collect(Collectors.toList());
+                validateNodeStates(mappingResponse,
+                                   dcReplication,
+                                   nodeNumber -> nodeNums.contains(nodeNumber) ? "Joining" : "Normal");
+
+                int nodeCount = annotation.nodesPerDc() * annotation.numDcs();
+                validateTokenRanges(mappingResponse, generateExpectedRanges(nodeCount));
+
+                validateReplicaMapping(mappingResponse, newNodes, expectedRangeMappings);
+                context.completeNow();
+            });
+        }
+        finally
+        {
+            for (int i = 0; i < (annotation.newNodesPerDc() * annotation.numDcs()); i++)
+            {
+                transientStateEnd.countDown();
+            }
+        }
+    }
+
+    private List<IUpgradeableInstance> startReplacementNodes(CountDownLatch nodeStart,
+                                                             UpgradeableCluster cluster,
+                                                             List<IUpgradeableInstance> nodesToRemove)
+    {
+        List<IUpgradeableInstance> newNodes = new ArrayList<>();
+        // Launch replacements nodes with the config of the removed nodes
+        for (IUpgradeableInstance removed : nodesToRemove)
+        {
+            // Add new instance for each removed instance as a replacement of its owned token
+            IInstanceConfig removedConfig = removed.config();
+            String remAddress = removedConfig.broadcastAddress().getAddress().getHostAddress();
+            int remPort = removedConfig.getInt("storage_port");
+            IUpgradeableInstance replacement =
+            ClusterUtils.addInstance(cluster, removedConfig,
+                                     c -> {
+                                         c.set("auto_bootstrap", true);
+                                         // explicitly DOES NOT set instances that failed startup as "shutdown"
+                                         // so subsequent attempts to shut down the instance are honored
+                                         c.set("dtest.api.startup.failure_as_shutdown", false);
+                                         c.with(Feature.GOSSIP,
+                                                Feature.JMX,
+                                                Feature.NATIVE_PROTOCOL);
+                                     });
+
+            new Thread(() -> ClusterUtils.start(replacement, (properties) -> {
+                properties.set(CassandraRelevantProperties.BOOTSTRAP_SKIP_SCHEMA_CHECK, true);
+                properties.set(CassandraRelevantProperties.BOOTSTRAP_SCHEMA_DELAY_MS,
+                               TimeUnit.SECONDS.toMillis(10L));
+                properties.with("cassandra.broadcast_interval_ms",
+                                Long.toString(TimeUnit.SECONDS.toMillis(30L)));
+                properties.with("cassandra.ring_delay_ms",
+                                Long.toString(TimeUnit.SECONDS.toMillis(10L)));
+                // This property tells cassandra that this new instance is replacing the node with
+                // address remAddress and port remPort
+                properties.with("cassandra.replace_address_first_boot", remAddress + ":" + remPort);
+            })).start();
+
+            Uninterruptibles.awaitUninterruptibly(nodeStart, 2, TimeUnit.MINUTES);
+            newNodes.add(replacement);
+        }
+        return newNodes;
+    }
+
+    private void stopNodes(IUpgradeableInstance seed, List<IUpgradeableInstance> removedNodes)
+    {
+        for (IUpgradeableInstance nodeToRemove : removedNodes)
+        {
+            ClusterUtils.stopUnchecked(nodeToRemove);
+            String remAddress = nodeToRemove.config().broadcastAddress().getAddress().getHostAddress();
+
+            List<ClusterUtils.RingInstanceDetails> ring = ClusterUtils.ring(seed);
+            List<ClusterUtils.RingInstanceDetails> match = ring.stream()
+                                                               .filter((d) -> d.getAddress().equals(remAddress))
+                                                               .collect(Collectors.toList());
+            assertThat(match.stream().anyMatch(r -> r.getStatus().equals("Down"))).isTrue();
+        }
+    }
+
+    private void validateReplicaMapping(TokenRangeReplicasResponse mappingResponse,
+                                        List<IUpgradeableInstance> newInstances,
+                                        Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    {
+        List<String> transientNodeAddresses = newInstances.stream().map(i -> {
+            InetSocketAddress address = i.config().broadcastAddress();
+            return address.getAddress().getHostAddress() +
+                   ":" +
+                   address.getPort();
+        }).collect(Collectors.toList());
+
+        Set<String> writeReplicaInstances = instancesFromReplicaSet(mappingResponse.writeReplicas());
+        Set<String> readReplicaInstances = instancesFromReplicaSet(mappingResponse.readReplicas());
+        assertThat(readReplicaInstances).doesNotContainAnyElementsOf(transientNodeAddresses);
+        assertThat(writeReplicaInstances).containsAll(transientNodeAddresses);
+
+        validateWriteReplicaMappings(mappingResponse.writeReplicas(), expectedRangeMappings);
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementMultiDCTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementMultiDCTest.java
new file mode 100644
index 0000000..30d4c42
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementMultiDCTest.java
@@ -0,0 +1,214 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Multi-DC Host replacement scenario integration tests for token range replica mapping endpoint with the in-jvm
+ * dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+class ReplacementMultiDCTest extends ReplacementBaseTest
+{
+    @CassandraIntegrationTest(
+    nodesPerDc = 5, newNodesPerDc = 1, numDcs = 2, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithNodeReplacementMultiDC(VertxTestContext context,
+                                                   ConfigurableCassandraTestContext cassandraTestContext)
+    throws Exception
+    {
+        BBHelperReplacementsMultiDC.reset();
+        UpgradeableCluster cluster = getMultiDCCluster(BBHelperReplacementsMultiDC::install, cassandraTestContext,
+                                                       builder -> builder.withDynamicPortAllocation(false));
+
+        List<IUpgradeableInstance> nodesToRemove = Arrays.asList(cluster.get(3), cluster.get(cluster.size()));
+        runReplacementTestScenario(context,
+                                   BBHelperReplacementsMultiDC.nodeStart,
+                                   BBHelperReplacementsMultiDC.transientStateStart,
+                                   BBHelperReplacementsMultiDC.transientStateEnd,
+                                   cluster,
+                                   nodesToRemove,
+                                   generateExpectedRangeMappingReplacementMultiDC());
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 10 node cluster
+     * across 2 DCs with the last 2 nodes leaving the cluster (1 per DC), with RF 3
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in
+     * subsequent ranges cascade with the next range excluding the first replica, and including the next replica from
+     * the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * In a multi-DC scenario, a single range will have nodes from both DCs. The replicas are grouped by DC here
+     * to allow per-DC validation as returned from the sidecar endpoint.
+     * <p>
+     * Ranges that including leaving node replicas will have [RF + no. leaving nodes in replica-set] replicas with
+     * the new replicas being the existing nodes in ring-order.
+     * <p>
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being the leaving node)
+     * Expected Range 2 - B, C, D, A (With A taking over the range of the leaving node)
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingReplacementMultiDC()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = annotation.nodesPerDc() * annotation.numDcs();
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges(nodeCount);
+        Map<Range<BigInteger>, List<String>> dc1Mapping = new HashMap<>();
+        Map<Range<BigInteger>, List<String>> dc2Mapping = new HashMap<>();
+
+        dc1Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        dc1Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.5", "127.0.0.7",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8"));
+
+        dc1Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.4", "127.0.0.6", "127.0.0.8"));
+
+        dc1Mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.5", "127.0.0.7", "127.0.0.9"));
+        dc2Mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.6", "127.0.0.8", "127.0.0.10",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.7", "127.0.0.9", "127.0.0.1"));
+        dc2Mapping.put(expectedRanges.get(6), Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.2",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(7), Arrays.asList("127.0.0.8", "127.0.0.10", "127.0.0.2",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.9", "127.0.0.1", "127.0.0.3",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(8), Arrays.asList("127.0.0.10", "127.0.0.2", "127.0.0.4",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5",
+                                                            "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(9), Arrays.asList("127.0.0.10", "127.0.0.2", "127.0.0.4",
+                                                            "127.0.0.12"));
+
+        dc1Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.1", "127.0.0.3", "127.0.0.5",
+                                                             "127.0.0.11"));
+        dc2Mapping.put(expectedRanges.get(10), Arrays.asList("127.0.0.2", "127.0.0.4", "127.0.0.6"));
+
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", dc1Mapping);
+                put("datacenter2", dc2Mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for multi-DC node replacement
+     */
+    @Shared
+    public static class BBHelperReplacementsMultiDC
+    {
+        // Additional latch used here to sequentially start the 2 new nodes to isolate the loading
+        // of the shared Cassandra system property REPLACE_ADDRESS_FIRST_BOOT across instances
+        static CountDownLatch nodeStart = new CountDownLatch(1);
+        static CountDownLatch transientStateStart = new CountDownLatch(2);
+        static CountDownLatch transientStateEnd = new CountDownLatch(2);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 10 node cluster (across 2 DCs) with a 2 replacement nodes
+            // We intercept the bootstrap of the replacement nodes to validate token ranges
+            if (nodeNumber > 10)
+            {
+                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(BBHelperReplacementsMultiDC.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();
+            nodeStart.countDown();
+            // trigger bootstrap start and wait until bootstrap is ready from test
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            nodeStart = new CountDownLatch(1);
+            transientStateStart = new CountDownLatch(2);
+            transientStateEnd = new CountDownLatch(2);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementTest.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementTest.java
new file mode 100644
index 0000000..c1e822f
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/ReplacementTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.tokenrange;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.function.BiConsumer;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.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;
+
+/**
+ * Host replacement scenario integration tests for token range replica mapping endpoint with the in-jvm dtest framework.
+ */
+@ExtendWith(VertxExtension.class)
+class ReplacementTest extends ReplacementBaseTest
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 1, network = true, gossip = true, buildCluster = false)
+    void retrieveMappingWithNodeReplacement(VertxTestContext context,
+                                            ConfigurableCassandraTestContext cassandraTestContext) throws Exception
+    {
+        BBHelperReplacementsNode.reset();
+        runReplacementTestScenario(context,
+                                   cassandraTestContext,
+                                   BBHelperReplacementsNode::install,
+                                   BBHelperReplacementsNode.nodeStart,
+                                   BBHelperReplacementsNode.transientStateStart,
+                                   BBHelperReplacementsNode.transientStateEnd,
+                                   generateExpectedRangeMappingNodeReplacement());
+    }
+
+    private void runReplacementTestScenario(VertxTestContext context,
+                                            ConfigurableCassandraTestContext cassandraTestContext,
+                                            BiConsumer<ClassLoader, Integer> instanceInitializer,
+                                            CountDownLatch nodeStart,
+                                            CountDownLatch transientStateStart,
+                                            CountDownLatch transientStateEnd,
+                                            Map<String, Map<Range<BigInteger>, List<String>>> expectedRangeMappings)
+    throws Exception
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        TokenSupplier tokenSupplier = TestTokenSupplier.evenlyDistributedTokens(annotation.nodesPerDc(),
+                                                                                annotation.newNodesPerDc(),
+                                                                                annotation.numDcs(),
+                                                                                1);
+        UpgradeableCluster cluster = cassandraTestContext.configureAndStartCluster(builder -> {
+            builder.withInstanceInitializer(instanceInitializer);
+            builder.withTokenSupplier(tokenSupplier);
+        });
+
+        List<IUpgradeableInstance> nodesToRemove = Collections.singletonList(cluster.get(cluster.size()));
+        runReplacementTestScenario(context,
+                                   nodeStart,
+                                   transientStateStart,
+                                   transientStateEnd,
+                                   cluster,
+                                   nodesToRemove,
+                                   expectedRangeMappings);
+    }
+
+    /**
+     * Generates expected token range and replica mappings specific to the test case involving a 5 node cluster
+     * with the last node replaced with a new node
+     * <p>
+     * Expected ranges are generated by adding RF replicas per range in increasing order. The replica-sets in
+     * subsequent ranges cascade with the next range excluding the first replica, and including the next replica from
+     * the nodes.
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D
+     * <p>
+     * Ranges will have [RF] replicas with ranges containing the replacement node having [RF + no. replacement nodes].
+     * <p>
+     * eg.
+     * Range 1 - A, B, C
+     * Range 2 - B, C, D (with D being replaced with E)
+     * Expected Range 2 - B, C, D, E (With E taking over the range of the node being replaced)
+     */
+    private Map<String, Map<Range<BigInteger>, List<String>>> generateExpectedRangeMappingNodeReplacement()
+    {
+        CassandraIntegrationTest annotation = sidecarTestContext.cassandraTestContext().annotation;
+        int nodeCount = annotation.nodesPerDc() * annotation.numDcs();
+        List<Range<BigInteger>> expectedRanges = generateExpectedRanges(nodeCount);
+        Map<Range<BigInteger>, List<String>> mapping = new HashMap<>();
+        mapping.put(expectedRanges.get(0), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        mapping.put(expectedRanges.get(1), Arrays.asList("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        mapping.put(expectedRanges.get(2), Arrays.asList("127.0.0.3", "127.0.0.4", "127.0.0.5",
+                                                         "127.0.0.6"));
+        mapping.put(expectedRanges.get(3), Arrays.asList("127.0.0.4", "127.0.0.5", "127.0.0.1",
+                                                         "127.0.0.6"));
+        mapping.put(expectedRanges.get(4), Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.2",
+                                                         "127.0.0.6"));
+        mapping.put(expectedRanges.get(5), Arrays.asList("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        return new HashMap<String, Map<Range<BigInteger>, List<String>>>()
+        {
+            {
+                put("datacenter1", mapping);
+            }
+        };
+    }
+
+    /**
+     * ByteBuddy helper for a single node replacement
+     */
+    @Shared
+    public static class BBHelperReplacementsNode
+    {
+        // Additional latch used here to sequentially start the 2 new nodes to isolate the loading
+        // of the shared Cassandra system property REPLACE_ADDRESS_FIRST_BOOT across instances
+        static CountDownLatch nodeStart = new CountDownLatch(1);
+        static CountDownLatch transientStateStart = new CountDownLatch(1);
+        static CountDownLatch transientStateEnd = new CountDownLatch(1);
+
+        public static void install(ClassLoader cl, Integer nodeNumber)
+        {
+            // Test case involves 5 node cluster with a replacement node
+            // We intercept the bootstrap of the replacement (6th) node 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(BBHelperReplacementsNode.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();
+            nodeStart.countDown();
+            // trigger bootstrap start and wait until bootstrap is ready from test
+            transientStateStart.countDown();
+            Uninterruptibles.awaitUninterruptibly(transientStateEnd);
+            return result;
+        }
+
+        public static void reset()
+        {
+            nodeStart = new CountDownLatch(1);
+            transientStateStart = new CountDownLatch(1);
+            transientStateEnd = new CountDownLatch(1);
+        }
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/TestTokenSupplier.java b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/TestTokenSupplier.java
new file mode 100644
index 0000000..1f8b07f
--- /dev/null
+++ b/src/test/integration/org/apache/cassandra/sidecar/routes/tokenrange/TestTokenSupplier.java
@@ -0,0 +1,110 @@
+/*
+ * 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.tokenrange;
+
+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 token supplier
+ */
+public class TestTokenSupplier
+{
+
+    /**
+     * Tokens are allocation used in tests to simulate token allocation nodes for an approx even distribution
+     * in a multiDC environment with neighboring nodes different DCs allocated adjacent tokens.
+     * Allocations for new nodes are interleaved among existing tokens.
+     * @param numNodesPerDC no. nodes from a single DC
+     * @param numDcs no. of datacenters
+     * @param numTokensPerNode no. tokens allocated to each node (this is always 1 if there are no vnodes)
+     * @return The token supplier that vends the tokens
+     */
+    static TokenSupplier evenlyDistributedTokens(int numNodesPerDC, int newNodesPerDC, int numDcs, int numTokensPerNode)
+    {
+        // Use token count using initial node count to first assign tokens to nodes
+        long totalTokens = (long) (numNodesPerDC) * numDcs * numTokensPerNode;
+        // Similar to Cassandra TokenSupplier, the increment is doubled to account for all tokens from MIN - MAX.
+        // For multi-DC, since neighboring nodes from different DCs have consecutive tokens, the increment is
+        // broadened by a factor of numDcs.
+        BigInteger increment = BigInteger.valueOf(((Long.MAX_VALUE / (totalTokens + 2)) * 2 * numDcs));
+        List<String>[] tokens = allocateExistingNodeTokens(numNodesPerDC,
+                                                           newNodesPerDC,
+                                                           numDcs,
+                                                           numTokensPerNode,
+                                                           increment);
+
+
+        // Initial value of the first new node
+        BigInteger value  = new BigInteger(tokens[0 + (numDcs - 1)].get(0));
+        BigInteger subIncrement = increment.divide(BigInteger.valueOf(2));
+
+        int nodeId = (int) totalTokens + 1;
+        for (int i = 0; i < numTokensPerNode; ++i)
+        {
+            while (nodeId <= ((numNodesPerDC + newNodesPerDC) * numDcs))
+            {
+                value = value.add(subIncrement);
+                // Nodes in different DCs are separated by a single token
+                for (int dc = 0; dc < numDcs; dc++)
+                {
+                    tokens[nodeId - 1].add(value.add(BigInteger.valueOf(dc)).toString());
+                    nodeId++;
+                }
+                value = value.add(subIncrement);
+            }
+        }
+
+        return (nodeIdx) -> tokens[nodeIdx - 1];
+    }
+
+    private static List<String>[] allocateExistingNodeTokens(int numNodesPerDC,
+                                                             int newNodesPerDC,
+                                                             int numDcs,
+                                                             int numTokensPerNode,
+                                                             BigInteger increment)
+    {
+        List<String>[] tokens = new List[(numNodesPerDC + newNodesPerDC) * numDcs];
+        for (int i = 0; i < ((numNodesPerDC + newNodesPerDC) * numDcs); ++i)
+        {
+            tokens[i] = new ArrayList(numTokensPerNode);
+        }
+
+        BigInteger value = BigInteger.valueOf(Long.MIN_VALUE + 1);
+
+        int nodeId = 1;
+        for (int i = 0; i < numTokensPerNode; ++i)
+        {
+            while (nodeId <= (numNodesPerDC * numDcs))
+            {
+                value = value.add(increment);
+                // Nodes in different DCs are separated by a single token
+                for (int dc = 0; dc < numDcs; dc++)
+                {
+                    tokens[nodeId - 1].add(value.add(BigInteger.valueOf(dc)).toString());
+                    nodeId++;
+                }
+            }
+        }
+        return tokens;
+    }
+}
diff --git a/src/test/integration/org/apache/cassandra/sidecar/testing/CassandraSidecarTestContext.java b/src/test/integration/org/apache/cassandra/sidecar/testing/CassandraSidecarTestContext.java
index d4a21a5..5243615 100644
--- a/src/test/integration/org/apache/cassandra/sidecar/testing/CassandraSidecarTestContext.java
+++ b/src/test/integration/org/apache/cassandra/sidecar/testing/CassandraSidecarTestContext.java
@@ -18,9 +18,7 @@
 
 package org.apache.cassandra.sidecar.testing;
 
-import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
@@ -45,55 +43,49 @@
 import org.apache.cassandra.sidecar.common.SimpleCassandraVersion;
 import org.apache.cassandra.sidecar.common.dns.DnsResolver;
 import org.apache.cassandra.sidecar.common.utils.SidecarVersionProvider;
-import org.apache.cassandra.testing.CassandraTestContext;
+import org.apache.cassandra.testing.AbstractCassandraTestContext;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Passed to integration tests.
  */
-public class CassandraSidecarTestContext extends CassandraTestContext
+public class CassandraSidecarTestContext implements AutoCloseable
 {
     public final SimpleCassandraVersion version;
-    public final UpgradeableCluster cluster;
-    public final InstancesConfig instancesConfig;
+    private final CassandraVersionProvider versionProvider;
+    private final DnsResolver dnsResolver;
+    private final AbstractCassandraTestContext abstractCassandraTestContext;
+    public InstancesConfig instancesConfig;
     private List<CQLSessionProvider> sessionProviders;
     private List<JmxClient> jmxClients;
     private static final SidecarVersionProvider svp = new SidecarVersionProvider("/sidecar.version");
+    private final List<InstanceConfigListener> instanceConfigListeners;
 
-    private CassandraSidecarTestContext(SimpleCassandraVersion version,
-                                        UpgradeableCluster cluster,
+    private CassandraSidecarTestContext(AbstractCassandraTestContext abstractCassandraTestContext,
+                                        SimpleCassandraVersion version,
                                         CassandraVersionProvider versionProvider,
-                                        DnsResolver dnsResolver) throws IOException
+                                        DnsResolver dnsResolver)
     {
-        super(org.apache.cassandra.testing.SimpleCassandraVersion.create(version.major,
-                                                                         version.minor,
-                                                                         version.patch), cluster);
+        this.instanceConfigListeners = new ArrayList<>();
+        this.abstractCassandraTestContext = abstractCassandraTestContext;
         this.version = version;
-        this.cluster = cluster;
-        this.sessionProviders = new ArrayList<>();
-        this.jmxClients = new ArrayList<>();
-        this.instancesConfig = buildInstancesConfig(versionProvider, dnsResolver);
+        this.versionProvider = versionProvider;
+        this.dnsResolver = dnsResolver;
     }
 
-    public static CassandraSidecarTestContext from(CassandraTestContext cassandraTestContext, DnsResolver dnsResolver)
+    public static CassandraSidecarTestContext from(AbstractCassandraTestContext cassandraTestContext,
+                                                   DnsResolver dnsResolver)
     {
         org.apache.cassandra.testing.SimpleCassandraVersion rootVersion = cassandraTestContext.version;
         SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(rootVersion.major,
                                                                              rootVersion.minor,
                                                                              rootVersion.patch);
         CassandraVersionProvider versionProvider = cassandraVersionProvider(dnsResolver);
-        try
-        {
-            return new CassandraSidecarTestContext(versionParsed,
-                                                   cassandraTestContext.getCluster(),
-                                                   versionProvider,
-                                                   dnsResolver);
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return new CassandraSidecarTestContext(cassandraTestContext,
+                                               versionParsed,
+                                               versionProvider,
+                                               dnsResolver);
     }
 
     public static CassandraVersionProvider cassandraVersionProvider(DnsResolver dnsResolver)
@@ -102,10 +94,117 @@
                .add(new CassandraFactory(dnsResolver, svp.sidecarVersion())).build();
     }
 
-    private InstancesConfig buildInstancesConfig(CassandraVersionProvider versionProvider,
-                                                 DnsResolver dnsResolver) throws IOException
+    public void registerInstanceConfigListener(InstanceConfigListener listener)
     {
+        this.instanceConfigListeners.add(listener);
+    }
+
+    public AbstractCassandraTestContext cassandraTestContext()
+    {
+        return abstractCassandraTestContext;
+    }
+
+    public boolean isClusterBuilt()
+    {
+        return abstractCassandraTestContext.cluster() != null;
+    }
+
+    public UpgradeableCluster cluster()
+    {
+        UpgradeableCluster cluster = abstractCassandraTestContext.cluster();
+        if (cluster == null)
+        {
+            throw new RuntimeException("The cluster must be built before it can be used");
+        }
+        return cluster;
+    }
+
+    public InstancesConfig instancesConfig()
+    {
+        if (instancesConfig == null
+            || instancesConfig.instances().size() != cluster().size()) // rebuild instances config if cluster changed
+        {
+            // clean-up any open sessions or client resources
+            close();
+            setInstancesConfig();
+        }
+        return this.instancesConfig;
+    }
+
+    public Session session()
+    {
+        return session(0);
+    }
+
+    public Session session(int instance)
+    {
+        if (sessionProviders == null)
+        {
+            setInstancesConfig();
+        }
+        return this.sessionProviders.get(instance).localCql();
+    }
+
+    @Override
+    public String toString()
+    {
+        return "CassandraTestContext{" +
+               ", version=" + version +
+               ", cluster=" + abstractCassandraTestContext.cluster() +
+               '}';
+    }
+
+    @Override
+    public void close()
+    {
+        if (sessionProviders != null)
+        {
+            sessionProviders.forEach(CQLSessionProvider::close);
+        }
+        if (instancesConfig != null)
+        {
+            instancesConfig.instances().forEach(instance -> instance.delegate().close());
+        }
+    }
+
+    public JmxClient jmxClient()
+    {
+        return jmxClient(0);
+    }
+
+    private JmxClient jmxClient(int instance)
+    {
+        if (jmxClients == null)
+        {
+            setInstancesConfig();
+        }
+        return jmxClients.get(instance);
+    }
+
+    /**
+     * A listener for {@link InstancesConfig} state changes
+     */
+    public interface InstanceConfigListener
+    {
+        void onInstancesConfigChange(InstancesConfig instancesConfig);
+    }
+
+    private void setInstancesConfig()
+    {
+        this.instancesConfig = buildInstancesConfig(versionProvider, dnsResolver);
+        for (InstanceConfigListener listener : instanceConfigListeners)
+        {
+            listener.onInstancesConfigChange(this.instancesConfig);
+        }
+    }
+
+    private InstancesConfig buildInstancesConfig(CassandraVersionProvider versionProvider,
+                                                 DnsResolver dnsResolver)
+    {
+        UpgradeableCluster cluster = cluster();
         List<InstanceMetadata> metadata = new ArrayList<>();
+        sessionProviders = new ArrayList<>();
+        jmxClients = new ArrayList<>();
         for (int i = 0; i < cluster.size(); i++)
         {
             IUpgradeableInstance instance = cluster.get(i + 1); // 1-based indexing to match node names;
@@ -116,17 +215,18 @@
                                                                            nativeTransportPort);
             CQLSessionProvider sessionProvider = new CQLSessionProvider(address, new NettyOptions());
             this.sessionProviders.add(sessionProvider);
-            JmxClient jmxClient = new JmxClient(hostName, config.jmxPort());
+            // The in-jvm dtest framework sometimes returns a cluster before all the jmx infrastructure is initialized.
+            // In these cases, we want to wait longer than the default retry/delay settings to connect.
+            JmxClient jmxClient = new JmxClient(hostName, config.jmxPort(), null, null, false, 20, 1000L);
             this.jmxClients.add(jmxClient);
 
             String[] dataDirectories = (String[]) config.get("data_file_directories");
             // Use the parent of the first data directory as the staging directory
             Path dataDirParentPath = Paths.get(dataDirectories[0]).getParent();
-            assertThat(dataDirParentPath).isNotNull()
-                                         .exists();
+            // If the cluster has not started yet, the node's root directory doesn't exist yet
+            assertThat(dataDirParentPath).isNotNull();
             Path stagingPath = dataDirParentPath.resolve("staging");
-            String uploadsStagingDirectory = stagingPath.toFile().getAbsolutePath();
-            Files.createDirectories(stagingPath);
+            String stagingDir = stagingPath.toFile().getAbsolutePath();
             CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(versionProvider,
                                                                              sessionProvider,
                                                                              jmxClient,
@@ -136,7 +236,7 @@
                                              .host(config.broadcastAddress().getAddress().getHostAddress())
                                              .port(nativeTransportPort)
                                              .dataDirs(Arrays.asList(dataDirectories))
-                                             .stagingDir(uploadsStagingDirectory)
+                                             .stagingDir(stagingDir)
                                              .delegate(delegate)
                                              .build());
         }
@@ -154,46 +254,4 @@
             return defaultValue;
         }
     }
-
-
-    public InstancesConfig getInstancesConfig()
-    {
-        return this.instancesConfig;
-    }
-
-    public Session session()
-    {
-        return session(0);
-    }
-
-    public Session session(int instance)
-    {
-        return this.sessionProviders.get(instance).localCql();
-    }
-
-    @Override
-    public String toString()
-    {
-        return "CassandraTestContext{" +
-               ", version=" + version +
-               ", cluster=" + cluster +
-               '}';
-    }
-
-    @Override
-    public void close()
-    {
-        sessionProviders.forEach(CQLSessionProvider::close);
-        instancesConfig.instances().forEach(instance -> instance.delegate().close());
-    }
-
-    public JmxClient jmxClient()
-    {
-        return jmxClient(0);
-    }
-
-    private JmxClient jmxClient(int instance)
-    {
-        return jmxClients.get(instance);
-    }
 }
diff --git a/src/test/integration/org/apache/cassandra/testing/AbstractCassandraTestContext.java b/src/test/integration/org/apache/cassandra/testing/AbstractCassandraTestContext.java
index 4c4cb96..fb33696 100644
--- a/src/test/integration/org/apache/cassandra/testing/AbstractCassandraTestContext.java
+++ b/src/test/integration/org/apache/cassandra/testing/AbstractCassandraTestContext.java
@@ -18,32 +18,58 @@
 
 package org.apache.cassandra.testing;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.shared.ShutdownException;
 
 /**
  * The base class for all CassandraTestContext implementations
  */
 public abstract class AbstractCassandraTestContext implements AutoCloseable
 {
+    private static final Logger LOGGER = LoggerFactory.getLogger(AbstractCassandraTestContext.class);
+
     public final SimpleCassandraVersion version;
     protected UpgradeableCluster cluster;
 
-    public AbstractCassandraTestContext(SimpleCassandraVersion version, UpgradeableCluster cluster)
+    public CassandraIntegrationTest annotation;
+
+    public AbstractCassandraTestContext(SimpleCassandraVersion version,
+                                        UpgradeableCluster cluster,
+                                        CassandraIntegrationTest annotation)
     {
         this.version = version;
         this.cluster = cluster;
+        this.annotation = annotation;
     }
 
-    public AbstractCassandraTestContext(SimpleCassandraVersion version)
+    public AbstractCassandraTestContext(SimpleCassandraVersion version,
+                                        CassandraIntegrationTest annotation)
     {
         this.version = version;
+        this.annotation = annotation;
     }
 
-    public void close() throws Exception
+    public UpgradeableCluster cluster()
+    {
+        return cluster;
+    }
+
+    @Override
+    public void close()
     {
         if (cluster != null)
         {
-            cluster.close();
+            try
+            {
+                cluster.close();
+            }
+            catch (ShutdownException shutdownException)
+            {
+                LOGGER.warn("Encountered shutdown exception which closing the cluster", shutdownException);
+            }
         }
     }
 }
diff --git a/src/test/integration/org/apache/cassandra/testing/CassandraTestContext.java b/src/test/integration/org/apache/cassandra/testing/CassandraTestContext.java
index 9cceb6a..d6fd852 100644
--- a/src/test/integration/org/apache/cassandra/testing/CassandraTestContext.java
+++ b/src/test/integration/org/apache/cassandra/testing/CassandraTestContext.java
@@ -29,9 +29,10 @@
 {
 
     public CassandraTestContext(SimpleCassandraVersion version,
-                                UpgradeableCluster cluster)
+                                UpgradeableCluster cluster,
+                                CassandraIntegrationTest annotation)
     {
-        super(version, cluster);
+        super(version, cluster, annotation);
     }
 
     @Override
diff --git a/src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java b/src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java
index c613be4..e07b84d 100644
--- a/src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java
+++ b/src/test/integration/org/apache/cassandra/testing/CassandraTestTemplate.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.testing;
 
 import java.lang.reflect.AnnotatedElement;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
@@ -40,6 +41,7 @@
 import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.TokenSupplier;
 import org.apache.cassandra.distributed.shared.Versions;
+import org.apache.cassandra.sidecar.common.utils.Preconditions;
 
 
 /**
@@ -54,7 +56,6 @@
  */
 public class CassandraTestTemplate implements TestTemplateInvocationContextProvider
 {
-
     private static final Logger LOGGER = LoggerFactory.getLogger(CassandraTestTemplate.class);
 
     private AbstractCassandraTestContext cassandraTestContext;
@@ -148,34 +149,21 @@
                 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
+                                  .withVersion(requestedVersion)
+                                  .withDCs(dcCount)
+                                  .withDataDirCount(annotation.numDataDirsPerInstance())
+                                  .withConfig(config -> annotationToFeatureList(annotation).forEach(config::with));
                 TokenSupplier tokenSupplier = TokenSupplier.evenlyDistributedTokens(finalNodeCount,
                                                                                     clusterBuilder.getTokenCount());
                 clusterBuilder.withTokenSupplier(tokenSupplier);
@@ -187,11 +175,13 @@
                     {
                         cluster.startup();
                     }
-                    cassandraTestContext = new CassandraTestContext(versionParsed, cluster);
+                    cassandraTestContext = new CassandraTestContext(versionParsed, cluster, annotation);
                 }
                 else
                 {
-                    cassandraTestContext = new ConfigurableCassandraTestContext(versionParsed, clusterBuilder);
+                    cassandraTestContext = new ConfigurableCassandraTestContext(versionParsed,
+                                                                                clusterBuilder,
+                                                                                annotation);
                 }
                 LOGGER.info("Testing {} against in-jvm dtest cluster", version);
                 LOGGER.info("Created Cassandra test context {}", cassandraTestContext);
@@ -214,6 +204,35 @@
         }
 
         /**
+         * Builds a list of configured {@link Feature features} requested in the {@link CassandraIntegrationTest}
+         * annotation.
+         *
+         * @param annotation the configured annotation
+         * @return a list of configured {@link Feature features}
+         */
+        private List<Feature> annotationToFeatureList(CassandraIntegrationTest annotation)
+        {
+            List<Feature> configuredFeatures = new ArrayList<>();
+            if (annotation.nativeTransport())
+            {
+                configuredFeatures.add(Feature.NATIVE_PROTOCOL);
+            }
+            if (annotation.jmx())
+            {
+                configuredFeatures.add(Feature.JMX);
+            }
+            if (annotation.gossip())
+            {
+                configuredFeatures.add(Feature.GOSSIP);
+            }
+            if (annotation.network())
+            {
+                configuredFeatures.add(Feature.NETWORK);
+            }
+            return configuredFeatures;
+        }
+
+        /**
          * Required for Junit to know the CassandraTestContext can be used in these tests
          *
          * @return a {@link ParameterResolver}
@@ -228,11 +247,15 @@
                 {
                     Class<?> parameterType = parameterContext.getParameter().getType();
                     CassandraIntegrationTest annotation =
-                        getCassandraIntegrationTestAnnotation(extensionContext, false);
+                    getCassandraIntegrationTestAnnotation(extensionContext, false);
                     if (annotation == null)
                     {
                         return false;
                     }
+                    if (parameterType.equals(AbstractCassandraTestContext.class))
+                    {
+                        return true;
+                    }
                     if (annotation.buildCluster())
                     {
                         if (parameterType.equals(CassandraTestContext.class))
diff --git a/src/test/integration/org/apache/cassandra/testing/ConfigurableCassandraTestContext.java b/src/test/integration/org/apache/cassandra/testing/ConfigurableCassandraTestContext.java
index b8ed0cf..9d4e0d5 100644
--- a/src/test/integration/org/apache/cassandra/testing/ConfigurableCassandraTestContext.java
+++ b/src/test/integration/org/apache/cassandra/testing/ConfigurableCassandraTestContext.java
@@ -30,15 +30,16 @@
 public class ConfigurableCassandraTestContext extends AbstractCassandraTestContext
 {
     public static final String BUILT_CLUSTER_CANNOT_BE_CONFIGURED_ERROR =
-        "Cannot configure a cluster after it is built. Please set the buildCluster annotation attribute to false, "
-        + "and do not call `getCluster` before calling this method.";
+    "Cannot configure a cluster after it is built. Please set the buildCluster annotation attribute to false, "
+    + "and do not call `getCluster` before calling this method.";
 
     private final UpgradeableCluster.Builder builder;
 
     public ConfigurableCassandraTestContext(SimpleCassandraVersion version,
-                                            UpgradeableCluster.Builder builder)
+                                            UpgradeableCluster.Builder builder,
+                                            CassandraIntegrationTest annotation)
     {
-        super(version);
+        super(version, annotation);
         this.builder = builder;
     }
 
@@ -54,7 +55,7 @@
     }
 
     public UpgradeableCluster configureAndStartCluster(Consumer<UpgradeableCluster.Builder> configurator)
-        throws IOException
+    throws IOException
     {
         cluster = configureCluster(configurator);
         cluster.startup();
diff --git a/src/test/java/org/apache/cassandra/sidecar/TestModule.java b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
index 3452b88..3e6ba6e 100644
--- a/src/test/java/org/apache/cassandra/sidecar/TestModule.java
+++ b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
@@ -42,6 +42,7 @@
 import org.apache.cassandra.sidecar.config.SslConfiguration;
 import org.apache.cassandra.sidecar.config.ThrottleConfiguration;
 import org.apache.cassandra.sidecar.config.yaml.HealthCheckConfigurationImpl;
+import org.apache.cassandra.sidecar.config.yaml.JmxConfigurationImpl;
 import org.apache.cassandra.sidecar.config.yaml.SSTableUploadConfigurationImpl;
 import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl;
 import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl;
@@ -80,7 +81,8 @@
         SSTableUploadConfiguration uploadConfiguration = new SSTableUploadConfigurationImpl(0F);
         ServiceConfiguration serviceConfiguration = new ServiceConfigurationImpl("127.0.0.1",
                                                                                  throttleConfiguration,
-                                                                                 uploadConfiguration);
+                                                                                 uploadConfiguration,
+                                                                                 new JmxConfigurationImpl());
         HealthCheckConfiguration healthCheckConfiguration = new HealthCheckConfigurationImpl(1000);
         return new SidecarConfigurationImpl(serviceConfiguration, sslConfiguration, healthCheckConfiguration);
     }
diff --git a/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java b/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
index ca055cd..26fe546 100644
--- a/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/ThrottleTest.java
@@ -57,7 +57,6 @@
     @BeforeEach
     void setUp() throws InterruptedException
     {
-
         Injector injector = Guice.createInjector(Modules.override(new MainModule()).with(new TestModule()));
         server = injector.getInstance(HttpServer.class);
         vertx = injector.getInstance(Vertx.class);
diff --git a/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java b/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
index 8d8ed00..83161f9 100644
--- a/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/config/SidecarConfigurationTest.java
@@ -93,6 +93,28 @@
     }
 
     @Test
+    void testReadingJmxConfiguration() throws IOException
+    {
+        Path yamlPath = yaml("config/sidecar_multiple_instances.yaml");
+        SidecarConfiguration config = SidecarConfigurationImpl.readYamlConfiguration(yamlPath);
+        assertThat(config.serviceConfiguration().jmxConfiguration()).isNotNull();
+        JmxConfiguration jmxConfiguration = config.serviceConfiguration().jmxConfiguration();
+        assertThat(jmxConfiguration.maxRetries()).isEqualTo(42);
+        assertThat(jmxConfiguration.retryDelayMillis()).isEqualTo(1234L);
+    }
+
+    @Test
+    void testReadingBlankJmxConfigurationReturnsDefaults() throws IOException
+    {
+        Path yamlPath = yaml("config/sidecar_missing_jmx.yaml");
+        SidecarConfiguration config = SidecarConfigurationImpl.readYamlConfiguration(yamlPath);
+        assertThat(config.serviceConfiguration().jmxConfiguration()).isNotNull();
+        JmxConfiguration jmxConfiguration = config.serviceConfiguration().jmxConfiguration();
+        assertThat(jmxConfiguration.maxRetries()).isEqualTo(3);
+        assertThat(jmxConfiguration.retryDelayMillis()).isEqualTo(200L);
+    }
+
+    @Test
     void testUploadsConfiguration() throws IOException
     {
         Path yamlPath = yaml("config/sidecar_multiple_instances.yaml");
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/GossipInfoHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/GossipInfoHandlerTest.java
index d7f8559..77b28d9 100644
--- a/src/test/java/org/apache/cassandra/sidecar/routes/GossipInfoHandlerTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/GossipInfoHandlerTest.java
@@ -65,7 +65,6 @@
     Vertx vertx;
     HttpServer server;
 
-    @SuppressWarnings("DataFlowIssue")
     @BeforeEach
     void before() throws InterruptedException
     {
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerTest.java
index 2c04242..4c34a49 100644
--- a/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/sstableuploads/SSTableImportHandlerTest.java
@@ -82,7 +82,8 @@
                   JsonObject error = response.bodyAsJsonObject();
                   assertThat(error.getInteger("code")).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
                   assertThat(error.getString("status")).isEqualTo("Bad Request");
-                  assertThat(error.getString("message")).isEqualTo("Invalid characters in keyspace: _n$ks_");
+                  assertThat(error.getString("message"))
+                  .isEqualTo("Invalid characters in keyspace: _n$ks_");
                   context.completeNow();
               })));
     }
@@ -98,7 +99,8 @@
                   JsonObject error = response.bodyAsJsonObject();
                   assertThat(error.getInteger("code")).isEqualTo(HttpResponseStatus.BAD_REQUEST.code());
                   assertThat(error.getString("status")).isEqualTo("Bad Request");
-                  assertThat(error.getString("message")).isEqualTo("Invalid characters in table name: _n$t_valid_");
+                  assertThat(error.getString("message"))
+                  .isEqualTo("Invalid characters in table name: _n$t_valid_");
                   context.completeNow();
               })));
     }
diff --git a/src/test/resources/config/sidecar_missing_jmx.yaml b/src/test/resources/config/sidecar_missing_jmx.yaml
new file mode 100644
index 0000000..ae37a11
--- /dev/null
+++ b/src/test/resources/config/sidecar_missing_jmx.yaml
@@ -0,0 +1,121 @@
+#
+# 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.
+#
+
+#
+# Cassandra SideCar configuration file
+#
+cassandra_instances:
+  - id: 1
+    host: localhost1
+    port: 9042
+    username: cassandra
+    password: cassandra
+    data_dirs:
+      - /ccm/test/node1/data0
+      - /ccm/test/node1/data1
+    staging_dir: /ccm/test/node1/sstable-staging
+    jmx_host: 127.0.0.1
+    jmx_port: 7100
+    jmx_ssl_enabled: false
+  #    jmx_role:
+  #    jmx_role_password:
+  - id: 2
+    host: localhost2
+    port: 9042
+    username: cassandra
+    password: cassandra
+    data_dirs:
+      - /ccm/test/node2/data0
+      - /ccm/test/node2/data1
+    staging_dir: /ccm/test/node2/sstable-staging
+    jmx_host: 127.0.0.1
+    jmx_port: 7200
+    jmx_ssl_enabled: false
+  #    jmx_role:
+  #    jmx_role_password:
+  - id: 3
+    host: localhost3
+    port: 9042
+    username: cassandra
+    password: cassandra
+    data_dirs:
+      - /ccm/test/node3/data0
+      - /ccm/test/node3/data1
+    staging_dir: /ccm/test/node3/sstable-staging
+    jmx_host: 127.0.0.1
+    jmx_port: 7300
+    jmx_ssl_enabled: false
+#    jmx_role:
+#    jmx_role_password:
+
+sidecar:
+  host: 0.0.0.0
+  port: 9043
+  request_idle_timeout_millis: 300000 # this field expects integer value
+  request_timeout_millis: 300000
+  throttle:
+    stream_requests_per_sec: 5000
+    delay_sec: 5
+    timeout_sec: 10
+  sstable_upload:
+    concurrent_upload_limit: 80
+    min_free_space_percent: 10
+    # file_permissions: "rw-r--r--" # when not specified, the default file permissions are owner read & write, group & others read
+  allowable_time_skew_in_minutes: 60
+  sstable_import:
+    poll_interval_millis: 100
+    cache:
+      expire_after_access_millis: 7200000 # 2 hours
+      maximum_size: 10000
+  worker_pools:
+    service:
+      name: "sidecar-worker-pool"
+      size: 20
+      max_execution_time_millis: 60000 # 60 seconds
+    internal:
+      name: "sidecar-internal-worker-pool"
+      size: 20
+      max_execution_time_millis: 900000 # 15 minutes
+#
+# Enable SSL configuration (Disabled by default)
+#
+#  ssl:
+#    enabled: true
+#    keystore:
+#      path: "path/to/keystore.p12"
+#      password: password
+#    truststore:
+#      path: "path/to/truststore.p12"
+#      password: password
+
+
+healthcheck:
+  poll_freq_millis: 30000
+
+cassandra_input_validation:
+  forbidden_keyspaces:
+    - system_schema
+    - system_traces
+    - system_distributed
+    - system
+    - system_auth
+    - system_views
+    - system_virtual_schema
+  allowed_chars_for_directory: "[a-zA-Z0-9_-]+"
+  allowed_chars_for_component_name: "[a-zA-Z0-9_-]+(.db|.cql|.json|.crc32|TOC.txt)"
+  allowed_chars_for_restricted_component_name: "[a-zA-Z0-9_-]+(.db|TOC.txt)"
diff --git a/src/test/resources/config/sidecar_multiple_instances.yaml b/src/test/resources/config/sidecar_multiple_instances.yaml
index 502b919..886b942 100644
--- a/src/test/resources/config/sidecar_multiple_instances.yaml
+++ b/src/test/resources/config/sidecar_multiple_instances.yaml
@@ -91,7 +91,9 @@
       name: "sidecar-internal-worker-pool"
       size: 20
       max_execution_time_millis: 900000 # 15 minutes
-
+  jmx:
+    max_retries: 42
+    retry_delay_millis: 1234
 #
 # Enable SSL configuration (Disabled by default)
 #
diff --git a/src/test/resources/config/sidecar_with_single_multiple_instances.yaml b/src/test/resources/config/sidecar_with_single_multiple_instances.yaml
index ec98f7d..a207884 100644
--- a/src/test/resources/config/sidecar_with_single_multiple_instances.yaml
+++ b/src/test/resources/config/sidecar_with_single_multiple_instances.yaml
@@ -39,7 +39,10 @@
     delay_sec: 5
     timeout_sec: 10
   allowable_time_skew_in_minutes: 60
-
+  jmx:
+    connection:
+      max_retries: 40
+      retry_delay_millis: 2000
 #
 # Enable SSL configuration (Disabled by default)
 #
diff --git a/vertx-client/build.gradle b/vertx-client/build.gradle
index ce02f60..d1d8516 100644
--- a/vertx-client/build.gradle
+++ b/vertx-client/build.gradle
@@ -35,6 +35,7 @@
 
 test {
     useJUnitPlatform()
+    maxParallelForks = Runtime.runtime.availableProcessors().intdiv(2) ?: 1
     reports {
         junitXml.enabled = true
         def destDir = Paths.get(rootProject.rootDir.absolutePath, "build", "test-results", "vertx-client").toFile()