yifan-c commented on code in PR #17:
URL: 
https://github.com/apache/cassandra-analytics/pull/17#discussion_r1419342286


##########
cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/expansion/JoiningDoubleClusterTest.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.analytics.expansion;
+
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.jupiter.api.TestInfo;
+
+import com.datastax.driver.core.ConsistencyLevel;
+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.analytics.TestUninterruptibles;
+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;
+
+
+public class JoiningDoubleClusterTest extends JoiningTestBase
+{
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 5, network = 
true, buildCluster = false)
+    void oneReadAllWrite(ConfigurableCassandraTestContext 
cassandraTestContext, TestInfo testInfo) throws Exception
+    {
+        BBHelperDoubleClusterSize.reset();
+        runJoiningTestScenario(cassandraTestContext,
+                               BBHelperDoubleClusterSize::install,
+                               BBHelperDoubleClusterSize.transientStateStart,
+                               BBHelperDoubleClusterSize.transientStateEnd,
+                               ConsistencyLevel.ONE,
+                               ConsistencyLevel.ALL,
+                               false,
+                               testInfo.getDisplayName());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 5, network = 
true, buildCluster = false)
+    void oneReadAllWriteFailure(ConfigurableCassandraTestContext 
cassandraTestContext, TestInfo testInfo) throws Exception
+    {
+        BBHelperDoubleClusterSizeFailure.reset();
+        runJoiningTestScenario(cassandraTestContext,
+                               BBHelperDoubleClusterSizeFailure::install,
+                               
BBHelperDoubleClusterSizeFailure.transientStateStart,
+                               
BBHelperDoubleClusterSizeFailure.transientStateEnd,
+                               ConsistencyLevel.ONE,
+                               ConsistencyLevel.ALL,
+                               true,
+                               testInfo.getDisplayName());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 5, network = 
true, buildCluster = false)
+    void quorumReadQuorumWrite(ConfigurableCassandraTestContext 
cassandraTestContext, TestInfo testInfo) throws Exception
+    {
+        BBHelperDoubleClusterSize.reset();
+        runJoiningTestScenario(cassandraTestContext,
+                               BBHelperDoubleClusterSize::install,
+                               BBHelperDoubleClusterSize.transientStateStart,
+                               BBHelperDoubleClusterSize.transientStateEnd,
+                               ConsistencyLevel.QUORUM,
+                               ConsistencyLevel.QUORUM,
+                               false,
+                               testInfo.getDisplayName());
+    }
+
+    @CassandraIntegrationTest(nodesPerDc = 5, newNodesPerDc = 5, network = 
true, buildCluster = false)
+    void quorumReadQuorumWriteFailure(ConfigurableCassandraTestContext 
cassandraTestContext, TestInfo testInfo) throws Exception
+    {
+        BBHelperDoubleClusterSizeFailure.reset();
+        runJoiningTestScenario(cassandraTestContext,
+                               BBHelperDoubleClusterSizeFailure::install,
+                               
BBHelperDoubleClusterSizeFailure.transientStateStart,
+                               
BBHelperDoubleClusterSizeFailure.transientStateEnd,
+                               ConsistencyLevel.QUORUM,
+                               ConsistencyLevel.QUORUM,
+                               true,
+                               testInfo.getDisplayName());
+    }
+
+    /**
+     * 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();
+            
TestUninterruptibles.awaitUninterruptiblyOrThrow(transientStateEnd, 2, 
TimeUnit.MINUTES);
+            return result;
+        }
+
+        public static void reset()
+        {
+            transientStateStart = new CountDownLatch(5);
+            transientStateEnd = new CountDownLatch(5);
+        }
+    }
+
+    /**
+     * ByteBuddy helper for doubling cluster size failure scenario
+     */
+    @Shared
+    public static class BBHelperDoubleClusterSizeFailure
+    {
+        static CountDownLatch transientStateStart = new CountDownLatch(5);

Review Comment:
   nit: rename to `transitionalStateStart`
   `transient` is a term can be confused with witness replica. 



##########
cassandra-analytics-integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/IntegrationTestBase.java:
##########
@@ -231,4 +273,84 @@ protected void waitForKeyspaceAndTable(String 
keyspaceName, String tableName)
         String.format("Keyspace/table %s/%s did not become visible on all 
sidecar instances",
                       keyspaceName, tableName));
     }
+
+    /**
+     * A {@link DnsResolver} instance used for tests that provides fast DNS 
resolution, to avoid blocking
+     * DNS resolution at the JDK/OS-level.
+     *
+     * <p><b>NOTE:</b> The resolver assumes that the addresses are of the form 
127.0.0.x, which is what is currently
+     * configured for integration tests.
+     */
+    static class FastDnsResolver implements DnsResolver

Review Comment:
   nit: `FastLocalhostResolver` to be more specific. 



##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java:
##########
@@ -339,140 +336,188 @@ public String getLowestCassandraVersion()
         return cassandraVersion;
     }
 
-    public String getVersionFromFeature()
+    @Override
+    public Map<RingInstance, InstanceAvailability> getInstanceAvailability()
     {
-        return null;
+        TokenRangeMapping<RingInstance> mapping = getTokenRangeMapping(true);
+        Map<RingInstance, InstanceAvailability> result =
+        mapping.getReplicaMetadata()
+               .stream()
+               .map(RingInstance::new)
+               .collect(Collectors.toMap(Function.identity(), 
this::determineInstanceAvailability));
+
+        if (LOGGER.isDebugEnabled())
+        {
+            result.forEach((inst, avail) -> LOGGER.debug("Instance {} has 
availability {}", inst, avail));
+        }
+        return result;
     }
 
-    protected List<NodeSettings> getAllNodeSettings()
+    private InstanceAvailability determineInstanceAvailability(RingInstance 
instance)
     {
-        List<NodeSettings> allNodeSettings = 
FutureUtils.bestEffortGet(allNodeSettingFutures,
-                                                                       
conf.getSidecarRequestMaxRetryDelayInSeconds(),
-                                                                       
TimeUnit.SECONDS);
-
-        if (allNodeSettings.isEmpty())
+        if (!instanceIsUp(instance.getRingInstance()))
         {
-            throw new RuntimeException(String.format("Unable to determine the 
node settings. 0/%d instances available.",
-                                                     
allNodeSettingFutures.size()));
+            return InstanceAvailability.UNAVAILABLE_DOWN;
         }
-        else if (allNodeSettings.size() < allNodeSettingFutures.size())
+        if (instanceIsBlocked(instance))
         {
-            LOGGER.warn("{}/{} instances were used to determine the node 
settings",
-                        allNodeSettings.size(), allNodeSettingFutures.size());
+            return InstanceAvailability.UNAVAILABLE_BLOCKED;
         }
-
-        return allNodeSettings;
-    }
-
-    public String getVersionFromSidecar()
-    {
-        NodeSettings nodeSettings = this.nodeSettings.get();
-        if (nodeSettings != null)
+        if (instanceIsNormal(instance.getRingInstance()) ||
+            instanceIsTransitioning(instance.getRingInstance()) ||
+            instanceIsBeingReplaced(instance.getRingInstance()))
         {
-            return nodeSettings.releaseVersion();
+            return InstanceAvailability.AVAILABLE;
         }
 
-        return getLowestVersion(getAllNodeSettings());
+        LOGGER.info("No valid state found for instance {}", instance);
+        // If it's not one of the above, it's inherently INVALID.
+        return InstanceAvailability.INVALID_STATE;
     }
 
-    protected RingResponse getRingResponse()
+    private TokenRangeMapping<RingInstance> getTokenRangeReplicas()
     {
-        RingResponse currentRingResponse = ringResponse;
-        if (currentRingResponse != null)
-        {
-            return currentRingResponse;
-        }
-
-        synchronized (this)
+        Map<String, Set<String>> writeReplicasByDC;
+        Map<String, Set<String>> pendingReplicasByDC;
+        Map<String, ReplicaMetadata> replicaMetadata;
+        Set<RingInstance> blockedInstances;
+        Set<RingInstance> replacementInstances;
+        Multimap<RingInstance, Range<BigInteger>> tokenRangesByInstance;
+        try
         {
-            if (ringResponse == null)
+            TokenRangeReplicasResponse response = 
getTokenRangesAndReplicaSets();
+            replicaMetadata = response.replicaMetadata();
+
+            tokenRangesByInstance = 
getTokenRangesByInstance(response.writeReplicas(), response.replicaMetadata());
+            LOGGER.info("Retrieved token ranges for {} instances from write 
replica set ",
+                        tokenRangesByInstance.size());
+
+            replacementInstances = response.replicaMetadata()
+                                           .values()
+                                           .stream()
+                                           .filter(m -> 
m.state().equalsIgnoreCase(InstanceState.REPLACING.toString()))
+                                           .map(RingInstance::new)
+                                           .collect(Collectors.toSet());
+
+            blockedInstances = response.replicaMetadata()
+                                       .values()
+                                       .stream()
+                                       .map(RingInstance::new)
+                                       .filter(this::instanceIsBlocked)
+                                       .collect(Collectors.toSet());
+
+            Set<String> blockedIps = blockedInstances.stream().map(i -> 
i.getRingInstance().address())
+                                                     
.collect(Collectors.toSet());
+
+            // Each token range has hosts by DC. We collate them across all 
ranges into all hosts by DC
+            writeReplicasByDC = response.writeReplicas()
+                                        .stream()
+                                        .flatMap(wr -> 
wr.replicasByDatacenter().entrySet().stream())
+                                        
.collect(Collectors.toMap(Map.Entry::getKey, e -> new HashSet<>(e.getValue()),
+                                                                  (l1, l2) -> 
filterAndMergeInstances(l1, l2, blockedIps)));
+
+            pendingReplicasByDC = getPendingReplicas(response, 
writeReplicasByDC);
+
+            if (LOGGER.isDebugEnabled())
             {
-                try
-                {
-                    ringResponse = getCurrentRingResponse();
-                }
-                catch (Exception exception)
-                {
-                    LOGGER.error("Failed to load Cassandra ring", exception);
-                    throw new RuntimeException(exception);
-                }
+                LOGGER.debug("Fetched token-ranges with dcs={}, 
write_replica_count={}, pending_replica_count={}",
+                             writeReplicasByDC.keySet(),
+                             
writeReplicasByDC.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()).size(),
+                             
pendingReplicasByDC.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()).size());
             }
-            return ringResponse;
         }
-    }
+        catch (ExecutionException | InterruptedException exception)
+        {
+            LOGGER.error("Failed to get token ranges, ", exception);
+            throw new RuntimeException(exception);
+        }
 
-    private RingResponse getCurrentRingResponse() throws Exception
-    {
-        return 
getCassandraContext().getSidecarClient().ring(conf.keyspace).get();
+        // Include availability info so CL checks can use it to exclude 
replacement hosts
+        return new TokenRangeMapping<>(getPartitioner(),
+                                       getReplicationFactor(),
+                                       writeReplicasByDC,
+                                       pendingReplicasByDC,
+                                       tokenRangesByInstance,
+                                       new 
ArrayList<>(replicaMetadata.values()),
+                                       blockedInstances,
+                                       replacementInstances);
     }
 
-    private static List<RingInstance> getSerializableInstances(RingResponse 
ringResponse)
+    private Set<String> filterAndMergeInstances(Set<String> instancesList1, 
Set<String> instancesList2, Set<String> blockedIPs)
     {
-        return ringResponse.stream()
-                           .map(RingInstance::new)
-                           .collect(Collectors.toList());
+        Set<String> merged = new HashSet<>();
+        // Removes blocked instances. If this is included, remove 
blockedInstances from CL checks
+        merged.addAll(instancesList1.stream().filter(i -> 
!blockedIPs.contains(i)).collect(Collectors.toSet()));
+        merged.addAll(instancesList2.stream().filter(i -> 
!blockedIPs.contains(i)).collect(Collectors.toSet()));
+
+        return merged;
     }
 
-    private static RingInstance getCasInstanceMethodsImpl(RingEntry ringEntry)
+    // Pending replicas are currently calculated by extracting the 
non-read-replicas from the write-replica-set
+    // This will be replaced by the instance state metadata when it is 
supported by the token-ranges API
+    private Map<String, Set<String>> 
getPendingReplicas(TokenRangeReplicasResponse response, Map<String, 
Set<String>> writeReplicasByDC)
     {
-        return new RingInstance(ringEntry);
+        Set<String> readReplicas = 
readReplicasFromTokenRangeResponse(response);
+        return writeReplicasByDC.entrySet()
+                                .stream()
+                                .filter(entry -> 
entry.getValue().stream().noneMatch(readReplicas::contains))
+                                .collect(Collectors.toMap(Map.Entry::getKey, 
Map.Entry::getValue));
     }
 
-    protected GossipInfoResponse getGossipInfo(boolean forceRefresh)
+    private Multimap<RingInstance, Range<BigInteger>> 
getTokenRangesByInstance(List<ReplicaInfo> writeReplicas,
+                                                                               
Map<String, ReplicaMetadata> replicaMetadata)
     {
-        GossipInfoResponse currentGossipInfo = gossipInfo;
-        if (!forceRefresh && currentGossipInfo != null)
-        {
-            return currentGossipInfo;
-        }
-
-        synchronized (this)
+        Multimap<RingInstance, Range<BigInteger>> instanceToRangeMap = 
ArrayListMultimap.create();
+        for (ReplicaInfo rInfo : writeReplicas)
         {
-            if (forceRefresh || gossipInfo == null)
+            Range<BigInteger> range = Range.openClosed(new 
BigInteger(rInfo.start()), new BigInteger(rInfo.end()));
+            for (Map.Entry<String, List<String>> dcReplicaEntry : 
rInfo.replicasByDatacenter().entrySet())
             {
-                try
-                {
-                    gossipInfo = 
cassandraContext.getSidecarClient().gossipInfo().get(conf.getHttpResponseTimeoutMs(),
-                                                                               
       TimeUnit.MILLISECONDS);
-                }
-                catch (ExecutionException | InterruptedException exception)
-                {
-                    LOGGER.error("Failed to retrieve gossip information");
-                    throw new RuntimeException("Failed to retrieve gossip 
information", exception);
-                }
-                catch (TimeoutException exception)
-                {
-                    Thread.currentThread().interrupt();
-                    throw new RuntimeException("Failed to retrieve gossip 
information", exception);
-                }
+                // For each writeReplica, get metadata and update map to 
include range
+                dcReplicaEntry.getValue().forEach(ipAddress -> {
+                    // Get metadata for this IP; Create RingInstance
+                    ReplicaMetadata replica = replicaMetadata.get(ipAddress);
+                    instanceToRangeMap.put(new RingInstance(replica), range);

Review Comment:
   What if `replicaMetadata.get(ipAddress)` returns null? How about add a check 
for the invalid state?



##########
cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/ResiliencyTestBase.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.analytics;
+
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import org.junit.jupiter.api.BeforeAll;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.ConsistencyLevel;
+import 
o.a.c.analytics.sidecar.shaded.testing.adapters.base.StorageJmxOperations;
+import o.a.c.analytics.sidecar.shaded.testing.common.JmxClient;
+import o.a.c.analytics.sidecar.shaded.testing.common.data.QualifiedTableName;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.testing.IntegrationTestBase;
+import org.apache.cassandra.spark.bulkwriter.DecoratedKey;
+import org.apache.cassandra.spark.bulkwriter.Tokenizer;
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import scala.Tuple2;
+
+import static junit.framework.TestCase.assertTrue;
+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;
+
+/**
+ * Base class for resiliency tests. Contains helper methods for data 
generation and validation
+ */
+public abstract class ResiliencyTestBase extends IntegrationTestBase
+{
+    public static final int rowCount = 1000;
+    protected static final String retrieveRows = "select * from " + 
TEST_KEYSPACE + ".%s";
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(ResiliencyTestBase.class);
+    private static final String createTableStmt = "create table if not exists 
%s (id int, course text, marks int, primary key (id));";
+
+    private final ExecutorService executorService = 
Executors.newCachedThreadPool();
+    private final AtomicReference<byte[]> errorOutput = new 
AtomicReference<>();
+    private final AtomicReference<byte[]> outputBytes = new 
AtomicReference<>();
+
+
+    public QualifiedTableName initializeSchema()
+    {
+        return initializeSchema(ImmutableMap.of("datacenter1", 1));
+    }
+
+    public QualifiedTableName initializeSchema(Map<String, Integer> rf)
+    {
+        createTestKeyspace(rf);
+        return createTestTable(createTableStmt);
+    }
+
+    public Set<String> getDataForRange(Range<BigInteger> range)
+    {
+        // Iterate through all data entries; filter only entries that belong 
to range; convert to strings
+        return generateExpectedData().stream()
+                                     .filter(t -> 
range.contains(t._1().getToken()))
+                                     .map(t -> t._2()[0] + ":" + t._2()[1] + 
":" + t._2()[2])
+                                     .collect(Collectors.toSet());
+    }
+
+    public List<Tuple2<DecoratedKey, Object[]>> generateExpectedData()
+    {
+        // "create table if not exists %s (id int, course text, marks int, 
primary key (id));";
+        List<ColumnType<?>> columnTypes = Arrays.asList(ColumnTypes.INT);
+        Tokenizer tokenizer = new Tokenizer(Arrays.asList(0),
+                                            Arrays.asList("id"),
+                                            columnTypes,
+                                            true
+        );
+        return IntStream.range(0, rowCount).mapToObj(recordNum -> {
+            Object[] columns = new Object[]
+                               {
+                               recordNum, "course" + recordNum, recordNum
+                               };
+            return Tuple2.apply(tokenizer.getDecoratedKey(columns), columns);
+        }).collect(Collectors.toList());
+    }
+
+    public Map<IUpgradeableInstance, Set<String>> 
getInstanceData(List<IUpgradeableInstance> instances,
+                                                                  boolean 
isPending)
+    {
+
+        return instances.stream().collect(Collectors.toMap(Function.identity(),
+                                                           i -> 
filterTokenRangeData(getRangesForInstance(i, isPending))));
+    }
+
+    public Set<String> filterTokenRangeData(List<Range<BigInteger>> ranges)
+    {
+        return ranges.stream()
+                     .map(this::getDataForRange)
+                     .flatMap(Collection::stream)
+                     .collect(Collectors.toSet());
+    }
+
+    /**
+     * Returns the expected set of rows as strings for each instance in the 
cluster
+     */
+    public Map<IUpgradeableInstance, Set<String>> 
generateExpectedInstanceData(UpgradeableCluster cluster,
+                                                                               
List<IUpgradeableInstance> pendingNodes)
+    {
+        List<IUpgradeableInstance> instances = 
cluster.stream().collect(Collectors.toList());
+        Map<IUpgradeableInstance, Set<String>> expectedInstanceData = 
getInstanceData(instances, false);
+        // Use pending ranges to get data for each transitioning instance
+        Map<IUpgradeableInstance, Set<String>> transitioningInstanceData = 
getInstanceData(pendingNodes, true);
+        expectedInstanceData.putAll(transitioningInstanceData.entrySet()
+                                                             .stream()
+                                                             .filter(e -> 
!e.getValue().isEmpty())
+                                                             
.collect(Collectors.toMap(Map.Entry::getKey,
+                                                                               
        Map.Entry::getValue)));
+        return expectedInstanceData;
+    }
+
+    public void validateData(String tableName, ConsistencyLevel cl)
+    {
+        String query = String.format(retrieveRows, tableName);
+        try
+        {
+            SimpleQueryResult resultSet = 
sidecarTestContext.cluster().get(1).coordinator()
+                                                            
.executeWithResult(query, mapConsistencyLevel(cl));
+            Set<String> rows = new HashSet<>();
+            for (SimpleQueryResult it = resultSet; it.hasNext();)
+            {
+                Row row = it.next();
+                if (row.get("id") == null || row.get("course") == null || 
row.get("marks") == null)
+                {
+                    throw new RuntimeException("Unrecognized row in table");
+                }
+
+                int id = row.getInteger("id");
+                String course = row.getString("course");
+                int marks = row.getInteger("marks");
+                rows.add(id + ":" + course + ":" + marks);
+            }
+            for (int i = 0; i < rowCount; i++)
+            {
+                String expectedRow = i + ":course" + i + ":" + i;
+                rows.remove(expectedRow);
+            }
+            assertTrue(rows.isEmpty());
+        }
+        catch (Exception ex)
+        {
+            logger.error("Validation Query failed", ex);
+            throw ex;
+        }
+    }
+
+    public void validateNodeSpecificData(QualifiedTableName table,
+                                         Map<IUpgradeableInstance, 
Set<String>> expectedInstanceData)
+    {
+        validateNodeSpecificData(table, expectedInstanceData, true);
+    }
+
+    public void validateNodeSpecificData(QualifiedTableName table,
+                                         Map<IUpgradeableInstance, 
Set<String>> expectedInstanceData,
+                                         boolean hasNewNodes)
+    {
+        for (IUpgradeableInstance instance : expectedInstanceData.keySet())
+        {
+            SimpleQueryResult qr = 
instance.executeInternalWithResult(String.format(retrieveRows, 
table.tableName()));
+            Set<String> rows = new HashSet<>();
+            while (qr.hasNext())
+            {
+                org.apache.cassandra.distributed.api.Row row = qr.next();
+                int id = row.getInteger("id");
+                String course = row.getString("course");
+                int marks = row.getInteger("marks");
+                rows.add(id + ":" + course + ":" + marks);
+            }
+
+            if (hasNewNodes)
+            {
+                
assertThat(rows).containsExactlyInAnyOrderElementsOf(expectedInstanceData.get(instance));
+            }
+            else
+            {
+                
assertThat(rows).containsAll(expectedInstanceData.get(instance));
+            }
+        }
+    }
+
+    public void closeStream(Closeable... closeables)
+    {
+        for (Closeable closeable : closeables)
+        {
+            try
+            {
+                if (closeable != null)
+                {
+                    closeable.close();
+                }
+            }
+            catch (IOException e)
+            {
+                LOGGER.error("Error closing " + closeable.toString(), e);
+            }
+        }
+    }
+
+    @BeforeAll
+    static void sparkSetup()
+    {
+
+    }
+

Review Comment:
   This method is meaningless. Can you remove it?



##########
cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/ResiliencyTestBase.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.analytics;
+
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import org.junit.jupiter.api.BeforeAll;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.ConsistencyLevel;
+import 
o.a.c.analytics.sidecar.shaded.testing.adapters.base.StorageJmxOperations;
+import o.a.c.analytics.sidecar.shaded.testing.common.JmxClient;
+import o.a.c.analytics.sidecar.shaded.testing.common.data.QualifiedTableName;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.testing.IntegrationTestBase;
+import org.apache.cassandra.spark.bulkwriter.DecoratedKey;
+import org.apache.cassandra.spark.bulkwriter.Tokenizer;
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import scala.Tuple2;
+
+import static junit.framework.TestCase.assertTrue;
+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;
+
+/**
+ * Base class for resiliency tests. Contains helper methods for data 
generation and validation
+ */
+public abstract class ResiliencyTestBase extends IntegrationTestBase
+{
+    public static final int rowCount = 1000;
+    protected static final String retrieveRows = "select * from " + 
TEST_KEYSPACE + ".%s";
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(ResiliencyTestBase.class);
+    private static final String createTableStmt = "create table if not exists 
%s (id int, course text, marks int, primary key (id));";
+
+    private final ExecutorService executorService = 
Executors.newCachedThreadPool();
+    private final AtomicReference<byte[]> errorOutput = new 
AtomicReference<>();
+    private final AtomicReference<byte[]> outputBytes = new 
AtomicReference<>();
+
+
+    public QualifiedTableName initializeSchema()
+    {
+        return initializeSchema(ImmutableMap.of("datacenter1", 1));
+    }
+
+    public QualifiedTableName initializeSchema(Map<String, Integer> rf)
+    {
+        createTestKeyspace(rf);
+        return createTestTable(createTableStmt);
+    }
+
+    public Set<String> getDataForRange(Range<BigInteger> range)
+    {
+        // Iterate through all data entries; filter only entries that belong 
to range; convert to strings
+        return generateExpectedData().stream()
+                                     .filter(t -> 
range.contains(t._1().getToken()))
+                                     .map(t -> t._2()[0] + ":" + t._2()[1] + 
":" + t._2()[2])
+                                     .collect(Collectors.toSet());
+    }
+
+    public List<Tuple2<DecoratedKey, Object[]>> generateExpectedData()
+    {
+        // "create table if not exists %s (id int, course text, marks int, 
primary key (id));";
+        List<ColumnType<?>> columnTypes = Arrays.asList(ColumnTypes.INT);
+        Tokenizer tokenizer = new Tokenizer(Arrays.asList(0),
+                                            Arrays.asList("id"),
+                                            columnTypes,
+                                            true
+        );
+        return IntStream.range(0, rowCount).mapToObj(recordNum -> {
+            Object[] columns = new Object[]
+                               {
+                               recordNum, "course" + recordNum, recordNum
+                               };
+            return Tuple2.apply(tokenizer.getDecoratedKey(columns), columns);
+        }).collect(Collectors.toList());
+    }
+
+    public Map<IUpgradeableInstance, Set<String>> 
getInstanceData(List<IUpgradeableInstance> instances,
+                                                                  boolean 
isPending)
+    {
+
+        return instances.stream().collect(Collectors.toMap(Function.identity(),
+                                                           i -> 
filterTokenRangeData(getRangesForInstance(i, isPending))));
+    }
+
+    public Set<String> filterTokenRangeData(List<Range<BigInteger>> ranges)
+    {
+        return ranges.stream()
+                     .map(this::getDataForRange)
+                     .flatMap(Collection::stream)
+                     .collect(Collectors.toSet());
+    }
+
+    /**
+     * Returns the expected set of rows as strings for each instance in the 
cluster
+     */
+    public Map<IUpgradeableInstance, Set<String>> 
generateExpectedInstanceData(UpgradeableCluster cluster,
+                                                                               
List<IUpgradeableInstance> pendingNodes)
+    {
+        List<IUpgradeableInstance> instances = 
cluster.stream().collect(Collectors.toList());
+        Map<IUpgradeableInstance, Set<String>> expectedInstanceData = 
getInstanceData(instances, false);
+        // Use pending ranges to get data for each transitioning instance
+        Map<IUpgradeableInstance, Set<String>> transitioningInstanceData = 
getInstanceData(pendingNodes, true);
+        expectedInstanceData.putAll(transitioningInstanceData.entrySet()
+                                                             .stream()
+                                                             .filter(e -> 
!e.getValue().isEmpty())
+                                                             
.collect(Collectors.toMap(Map.Entry::getKey,
+                                                                               
        Map.Entry::getValue)));
+        return expectedInstanceData;
+    }
+
+    public void validateData(String tableName, ConsistencyLevel cl)
+    {
+        String query = String.format(retrieveRows, tableName);
+        try
+        {
+            SimpleQueryResult resultSet = 
sidecarTestContext.cluster().get(1).coordinator()
+                                                            
.executeWithResult(query, mapConsistencyLevel(cl));
+            Set<String> rows = new HashSet<>();
+            for (SimpleQueryResult it = resultSet; it.hasNext();)
+            {
+                Row row = it.next();
+                if (row.get("id") == null || row.get("course") == null || 
row.get("marks") == null)
+                {
+                    throw new RuntimeException("Unrecognized row in table");
+                }
+
+                int id = row.getInteger("id");
+                String course = row.getString("course");
+                int marks = row.getInteger("marks");
+                rows.add(id + ":" + course + ":" + marks);
+            }
+            for (int i = 0; i < rowCount; i++)
+            {
+                String expectedRow = i + ":course" + i + ":" + i;
+                rows.remove(expectedRow);
+            }
+            assertTrue(rows.isEmpty());
+        }
+        catch (Exception ex)
+        {
+            logger.error("Validation Query failed", ex);
+            throw ex;
+        }
+    }
+
+    public void validateNodeSpecificData(QualifiedTableName table,
+                                         Map<IUpgradeableInstance, 
Set<String>> expectedInstanceData)
+    {
+        validateNodeSpecificData(table, expectedInstanceData, true);
+    }
+
+    public void validateNodeSpecificData(QualifiedTableName table,
+                                         Map<IUpgradeableInstance, 
Set<String>> expectedInstanceData,
+                                         boolean hasNewNodes)
+    {
+        for (IUpgradeableInstance instance : expectedInstanceData.keySet())
+        {
+            SimpleQueryResult qr = 
instance.executeInternalWithResult(String.format(retrieveRows, 
table.tableName()));
+            Set<String> rows = new HashSet<>();
+            while (qr.hasNext())
+            {
+                org.apache.cassandra.distributed.api.Row row = qr.next();
+                int id = row.getInteger("id");
+                String course = row.getString("course");
+                int marks = row.getInteger("marks");
+                rows.add(id + ":" + course + ":" + marks);
+            }
+
+            if (hasNewNodes)
+            {
+                
assertThat(rows).containsExactlyInAnyOrderElementsOf(expectedInstanceData.get(instance));
+            }
+            else
+            {
+                
assertThat(rows).containsAll(expectedInstanceData.get(instance));
+            }
+        }
+    }
+
+    public void closeStream(Closeable... closeables)
+    {
+        for (Closeable closeable : closeables)
+        {
+            try
+            {
+                if (closeable != null)
+                {
+                    closeable.close();
+                }
+            }
+            catch (IOException e)
+            {
+                LOGGER.error("Error closing " + closeable.toString(), e);
+            }
+        }
+    }
+
+    @BeforeAll
+    static void sparkSetup()
+    {
+
+    }
+
+    private static String getCleanedClasspath()
+    {
+        String classpath = System.getProperty("java.class.path");
+        Pattern pattern = 
Pattern.compile(":?[^:]*/dtest-\\d\\.\\d+\\.\\d+\\.jar:?");
+        return pattern.matcher(classpath).replaceAll(":");
+    }
+
+    private List<Range<BigInteger>> getRangesForInstance(IUpgradeableInstance 
instance, boolean isPending)
+    {
+        IInstanceConfig config = instance.config();
+        JmxClient client = JmxClient.builder()
+                                    
.host(config.broadcastAddress().getAddress().getHostAddress())
+                                    .port(config.jmxPort())
+                                    .build();
+        StorageJmxOperations ss = client.proxy(StorageJmxOperations.class, 
"org.apache.cassandra.db:type=StorageService");
+
+        Map<List<String>, List<String>> ranges = isPending ? 
ss.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)
+                                                           : 
ss.getRangeToEndpointWithPortMap(TEST_KEYSPACE);
+
+        // filter ranges that belong to the instance
+        return ranges.entrySet()
+                     .stream()
+                     .filter(e -> 
e.getValue().contains(instance.broadcastAddress().getAddress().getHostAddress()
+                                                        + ":" + 
instance.broadcastAddress().getPort()))
+                     .map(e -> unwrapRanges(e.getKey()))
+                     .flatMap(Collection::stream)
+                     .collect(Collectors.toList());
+    }
+
+    private List<Range<BigInteger>> unwrapRanges(List<String> range)
+    {
+        List<Range<BigInteger>> ranges = new ArrayList<Range<BigInteger>>();
+        BigInteger start = new BigInteger(range.get(0));
+        BigInteger end = new BigInteger(range.get(1));
+        if (start.compareTo(end) > 0)
+        {
+            ranges.add(Range.openClosed(start, 
BigInteger.valueOf(Long.MAX_VALUE)));
+            ranges.add(Range.openClosed(BigInteger.valueOf(Long.MIN_VALUE), 
end));
+        }
+        else
+        {
+            ranges.add(Range.openClosed(start, end));
+        }
+        return ranges;
+    }
+
+    private org.apache.cassandra.distributed.api.ConsistencyLevel 
mapConsistencyLevel(ConsistencyLevel cl)
+    {
+        return 
org.apache.cassandra.distributed.api.ConsistencyLevel.valueOf(cl.name());
+    }
+
+    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 QualifiedTableName bulkWriteData(ConsistencyLevel writeCL, 
String testName)
+    throws InterruptedException
+    {
+        CassandraIntegrationTest annotation = 
sidecarTestContext.cassandraTestContext().annotation;
+        List<String> sidecarInstances = generateSidecarInstances();
+
+        ImmutableMap<String, Integer> rf;
+        if (annotation.numDcs() > 1 && annotation.useCrossDcKeyspace())
+        {
+            rf = ImmutableMap.of("datacenter1", DEFAULT_RF, "datacenter2", 
DEFAULT_RF);
+        }
+        else
+        {
+            rf = ImmutableMap.of("datacenter1", DEFAULT_RF);
+        }
+
+        QualifiedTableName schema = initializeSchema(rf);
+        Thread.sleep(2000);
+        List<String> command = new ArrayList<>();
+        command.add(System.getProperty("java.home") + File.separator + "bin" + 
File.separator + "java");
+        // Uncomment the line below to debug on localhost
+        // 
command.add("-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=*:5151");
+        command.addAll(ManagementFactory.getRuntimeMXBean().getInputArguments()
+                                        .stream()
+                                        // Remove any already-existing 
debugger agents from the arguments
+                                        .filter(s -> shouldRetainSetting(s))
+                                        .collect(Collectors.toList()));
+        command.add("-Dspark.cassandra_analytics.request.max_connections=5");
+        // Set both max and min heap sizes because otherwise
+        command.add("-Xmx512m");
+        command.add("-Xms512m");
+        command.add("-XX:MaxDirectMemorySize=128m");
+        command.add("-cp");
+        String cleanedClasspath = getCleanedClasspath();
+        command.add(cleanedClasspath);
+        command.add(RunWriteJob.class.getName());
+        command.add(String.join(",", sidecarInstances));
+        command.add(schema.keyspace());
+        command.add(schema.tableName());
+        command.add(writeCL.name());
+        command.add(String.valueOf(server.actualPort()));
+        command.add(String.valueOf(rowCount));
+        try
+        {
+            ProcessBuilder builder = new ProcessBuilder(command);
+            LOGGER.info("Running: {}", String.join("\n", command));
+            Process process = builder.start();
+            CountDownLatch finishLatch = startReadOutputThreads(process, 
String.join(" ", command));
+            finishLatch.await(); // TODO: Timeout? - can call 
`process.destroy` if it hangs
+            int exitCode = process.waitFor();
+            if (exitCode != 0)
+            {
+                String stdout = new String(outputBytes.get());
+                String stdErr = new String(errorOutput.get());
+                LOGGER.error("Spark STDOUT:\n*****{}\n*****", stdout);

Review Comment:
   Should it use `info` for stdout?



##########
cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/ResiliencyTestBase.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * 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.analytics;
+
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
+import java.math.BigInteger;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import org.junit.jupiter.api.BeforeAll;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.ConsistencyLevel;
+import 
o.a.c.analytics.sidecar.shaded.testing.adapters.base.StorageJmxOperations;
+import o.a.c.analytics.sidecar.shaded.testing.common.JmxClient;
+import o.a.c.analytics.sidecar.shaded.testing.common.data.QualifiedTableName;
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.IInstanceConfig;
+import org.apache.cassandra.distributed.api.IUpgradeableInstance;
+import org.apache.cassandra.distributed.api.Row;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.api.TokenSupplier;
+import org.apache.cassandra.sidecar.testing.IntegrationTestBase;
+import org.apache.cassandra.spark.bulkwriter.DecoratedKey;
+import org.apache.cassandra.spark.bulkwriter.Tokenizer;
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
+import scala.Tuple2;
+
+import static junit.framework.TestCase.assertTrue;
+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;
+
+/**
+ * Base class for resiliency tests. Contains helper methods for data 
generation and validation
+ */
+public abstract class ResiliencyTestBase extends IntegrationTestBase
+{
+    public static final int rowCount = 1000;
+    protected static final String retrieveRows = "select * from " + 
TEST_KEYSPACE + ".%s";
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(ResiliencyTestBase.class);
+    private static final String createTableStmt = "create table if not exists 
%s (id int, course text, marks int, primary key (id));";
+
+    private final ExecutorService executorService = 
Executors.newCachedThreadPool();
+    private final AtomicReference<byte[]> errorOutput = new 
AtomicReference<>();
+    private final AtomicReference<byte[]> outputBytes = new 
AtomicReference<>();
+
+
+    public QualifiedTableName initializeSchema()
+    {
+        return initializeSchema(ImmutableMap.of("datacenter1", 1));
+    }
+
+    public QualifiedTableName initializeSchema(Map<String, Integer> rf)
+    {
+        createTestKeyspace(rf);
+        return createTestTable(createTableStmt);
+    }
+
+    public Set<String> getDataForRange(Range<BigInteger> range)
+    {
+        // Iterate through all data entries; filter only entries that belong 
to range; convert to strings
+        return generateExpectedData().stream()
+                                     .filter(t -> 
range.contains(t._1().getToken()))
+                                     .map(t -> t._2()[0] + ":" + t._2()[1] + 
":" + t._2()[2])
+                                     .collect(Collectors.toSet());
+    }
+
+    public List<Tuple2<DecoratedKey, Object[]>> generateExpectedData()
+    {
+        // "create table if not exists %s (id int, course text, marks int, 
primary key (id));";
+        List<ColumnType<?>> columnTypes = Arrays.asList(ColumnTypes.INT);
+        Tokenizer tokenizer = new Tokenizer(Arrays.asList(0),
+                                            Arrays.asList("id"),
+                                            columnTypes,
+                                            true
+        );
+        return IntStream.range(0, rowCount).mapToObj(recordNum -> {
+            Object[] columns = new Object[]
+                               {
+                               recordNum, "course" + recordNum, recordNum
+                               };
+            return Tuple2.apply(tokenizer.getDecoratedKey(columns), columns);
+        }).collect(Collectors.toList());
+    }
+
+    public Map<IUpgradeableInstance, Set<String>> 
getInstanceData(List<IUpgradeableInstance> instances,
+                                                                  boolean 
isPending)
+    {
+
+        return instances.stream().collect(Collectors.toMap(Function.identity(),
+                                                           i -> 
filterTokenRangeData(getRangesForInstance(i, isPending))));
+    }
+
+    public Set<String> filterTokenRangeData(List<Range<BigInteger>> ranges)
+    {
+        return ranges.stream()
+                     .map(this::getDataForRange)
+                     .flatMap(Collection::stream)
+                     .collect(Collectors.toSet());
+    }
+
+    /**
+     * Returns the expected set of rows as strings for each instance in the 
cluster
+     */
+    public Map<IUpgradeableInstance, Set<String>> 
generateExpectedInstanceData(UpgradeableCluster cluster,
+                                                                               
List<IUpgradeableInstance> pendingNodes)
+    {
+        List<IUpgradeableInstance> instances = 
cluster.stream().collect(Collectors.toList());
+        Map<IUpgradeableInstance, Set<String>> expectedInstanceData = 
getInstanceData(instances, false);
+        // Use pending ranges to get data for each transitioning instance
+        Map<IUpgradeableInstance, Set<String>> transitioningInstanceData = 
getInstanceData(pendingNodes, true);
+        expectedInstanceData.putAll(transitioningInstanceData.entrySet()
+                                                             .stream()
+                                                             .filter(e -> 
!e.getValue().isEmpty())
+                                                             
.collect(Collectors.toMap(Map.Entry::getKey,
+                                                                               
        Map.Entry::getValue)));
+        return expectedInstanceData;
+    }
+
+    public void validateData(String tableName, ConsistencyLevel cl)
+    {
+        String query = String.format(retrieveRows, tableName);
+        try
+        {
+            SimpleQueryResult resultSet = 
sidecarTestContext.cluster().get(1).coordinator()
+                                                            
.executeWithResult(query, mapConsistencyLevel(cl));
+            Set<String> rows = new HashSet<>();
+            for (SimpleQueryResult it = resultSet; it.hasNext();)
+            {
+                Row row = it.next();
+                if (row.get("id") == null || row.get("course") == null || 
row.get("marks") == null)
+                {
+                    throw new RuntimeException("Unrecognized row in table");
+                }
+
+                int id = row.getInteger("id");
+                String course = row.getString("course");
+                int marks = row.getInteger("marks");
+                rows.add(id + ":" + course + ":" + marks);
+            }
+            for (int i = 0; i < rowCount; i++)
+            {
+                String expectedRow = i + ":course" + i + ":" + i;
+                rows.remove(expectedRow);
+            }
+            assertTrue(rows.isEmpty());
+        }
+        catch (Exception ex)
+        {
+            logger.error("Validation Query failed", ex);
+            throw ex;
+        }
+    }
+
+    public void validateNodeSpecificData(QualifiedTableName table,
+                                         Map<IUpgradeableInstance, 
Set<String>> expectedInstanceData)
+    {
+        validateNodeSpecificData(table, expectedInstanceData, true);
+    }
+
+    public void validateNodeSpecificData(QualifiedTableName table,
+                                         Map<IUpgradeableInstance, 
Set<String>> expectedInstanceData,
+                                         boolean hasNewNodes)
+    {
+        for (IUpgradeableInstance instance : expectedInstanceData.keySet())
+        {
+            SimpleQueryResult qr = 
instance.executeInternalWithResult(String.format(retrieveRows, 
table.tableName()));
+            Set<String> rows = new HashSet<>();
+            while (qr.hasNext())
+            {
+                org.apache.cassandra.distributed.api.Row row = qr.next();
+                int id = row.getInteger("id");
+                String course = row.getString("course");
+                int marks = row.getInteger("marks");
+                rows.add(id + ":" + course + ":" + marks);
+            }
+
+            if (hasNewNodes)
+            {
+                
assertThat(rows).containsExactlyInAnyOrderElementsOf(expectedInstanceData.get(instance));
+            }
+            else
+            {
+                
assertThat(rows).containsAll(expectedInstanceData.get(instance));
+            }
+        }
+    }
+
+    public void closeStream(Closeable... closeables)
+    {
+        for (Closeable closeable : closeables)
+        {
+            try
+            {
+                if (closeable != null)
+                {
+                    closeable.close();
+                }
+            }
+            catch (IOException e)
+            {
+                LOGGER.error("Error closing " + closeable.toString(), e);
+            }
+        }
+    }
+
+    @BeforeAll
+    static void sparkSetup()
+    {
+
+    }
+
+    private static String getCleanedClasspath()
+    {
+        String classpath = System.getProperty("java.class.path");
+        Pattern pattern = 
Pattern.compile(":?[^:]*/dtest-\\d\\.\\d+\\.\\d+\\.jar:?");
+        return pattern.matcher(classpath).replaceAll(":");
+    }
+
+    private List<Range<BigInteger>> getRangesForInstance(IUpgradeableInstance 
instance, boolean isPending)
+    {
+        IInstanceConfig config = instance.config();
+        JmxClient client = JmxClient.builder()
+                                    
.host(config.broadcastAddress().getAddress().getHostAddress())
+                                    .port(config.jmxPort())
+                                    .build();
+        StorageJmxOperations ss = client.proxy(StorageJmxOperations.class, 
"org.apache.cassandra.db:type=StorageService");
+
+        Map<List<String>, List<String>> ranges = isPending ? 
ss.getPendingRangeToEndpointWithPortMap(TEST_KEYSPACE)
+                                                           : 
ss.getRangeToEndpointWithPortMap(TEST_KEYSPACE);
+
+        // filter ranges that belong to the instance
+        return ranges.entrySet()
+                     .stream()
+                     .filter(e -> 
e.getValue().contains(instance.broadcastAddress().getAddress().getHostAddress()
+                                                        + ":" + 
instance.broadcastAddress().getPort()))
+                     .map(e -> unwrapRanges(e.getKey()))
+                     .flatMap(Collection::stream)
+                     .collect(Collectors.toList());
+    }
+
+    private List<Range<BigInteger>> unwrapRanges(List<String> range)
+    {
+        List<Range<BigInteger>> ranges = new ArrayList<Range<BigInteger>>();
+        BigInteger start = new BigInteger(range.get(0));
+        BigInteger end = new BigInteger(range.get(1));
+        if (start.compareTo(end) > 0)
+        {
+            ranges.add(Range.openClosed(start, 
BigInteger.valueOf(Long.MAX_VALUE)));
+            ranges.add(Range.openClosed(BigInteger.valueOf(Long.MIN_VALUE), 
end));
+        }
+        else
+        {
+            ranges.add(Range.openClosed(start, end));
+        }
+        return ranges;
+    }
+
+    private org.apache.cassandra.distributed.api.ConsistencyLevel 
mapConsistencyLevel(ConsistencyLevel cl)
+    {
+        return 
org.apache.cassandra.distributed.api.ConsistencyLevel.valueOf(cl.name());
+    }
+
+    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 QualifiedTableName bulkWriteData(ConsistencyLevel writeCL, 
String testName)
+    throws InterruptedException
+    {
+        CassandraIntegrationTest annotation = 
sidecarTestContext.cassandraTestContext().annotation;
+        List<String> sidecarInstances = generateSidecarInstances();
+
+        ImmutableMap<String, Integer> rf;
+        if (annotation.numDcs() > 1 && annotation.useCrossDcKeyspace())
+        {
+            rf = ImmutableMap.of("datacenter1", DEFAULT_RF, "datacenter2", 
DEFAULT_RF);
+        }
+        else
+        {
+            rf = ImmutableMap.of("datacenter1", DEFAULT_RF);
+        }
+
+        QualifiedTableName schema = initializeSchema(rf);
+        Thread.sleep(2000);
+        List<String> command = new ArrayList<>();

Review Comment:
   Can you add a comment to explain why running the spark job in a sub-process? 
I assume it is for resource isolation. 



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

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

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to