You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yc...@apache.org on 2022/10/17 21:24:08 UTC

[cassandra-sidecar] branch trunk updated: CASSANDRASC-46: Migrate minikube to testcontainers for integration tests

This is an automated email from the ASF dual-hosted git repository.

ycai pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-sidecar.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 05c0bbe  CASSANDRASC-46: Migrate minikube to testcontainers for integration tests
05c0bbe is described below

commit 05c0bbe29f75d678596af09abb0c68d15e93f7ba
Author: Francisco Guerrero <fr...@apple.com>
AuthorDate: Fri Oct 14 11:48:23 2022 -0700

    CASSANDRASC-46: Migrate minikube to testcontainers for integration tests
    
    The existing Cassandra Sidecar integration testing suite uses Minikube to provision a
    Cassandra service and it performs tests against the running database. This database runs
    on Minikube.
    
    A new alternative is to use [testcontainers](https://www.testcontainers.org), which requires
    Docker to run tests. The concept is similar, but the benefit is that *testcontainers* is
    well integrated into the java ecosystem and it works well with junit5. By replacing Minikube
    with `testcontainers` we can simplify the setup process and reduce the complexity for running
    integration tests.
    
    Additionally, `testcontainers` is supported as part of the testing infrastructure inside
    [CircleCI](https://www.testcontainers.org/supported_docker_environment/continuous_integration/circle_ci/).
    Currently, our Minikube tests are broken both locally and in CircleCI. Moving to `testcontainers`
    would also unblock us for further development.
    
    Minor fix when running testcontainers
    
    patch by Francisco Guerrero; reviewed by Yifan Cai, Dinesh Joshi for CASSANDRASC-46
---
 README.md                                          |  43 +--
 build.gradle                                       |  48 +--
 cassandra-integration-tests/build.gradle           |  34 +-
 .../sidecar/common/{testing => }/DelegateTest.java |  26 +-
 .../cassandra/sidecar/common/StatusTest.java       |  13 +-
 .../sidecar/common/testing/CassandraPod.java       | 364 ---------------------
 .../common/testing/CassandraPodException.java      |  30 --
 .../common/testing/CassandraTestContext.java       |   5 +-
 .../common/testing/CassandraTestTemplate.java      |  20 +-
 .../common/testing/TestVersionSupplier.java        |   6 +-
 .../src/test/resources/logback-test.xml            |  12 +
 .../sidecar/cassandra40/Cassandra40Factory.java    |  10 +-
 common/build.gradle                                |   2 +-
 containers/build.gradle                            |  76 -----
 containers/src/Cassandra40/Dockerfile              |  19 --
 containers/src/docker-entrypoint.sh                |  15 -
 containers/src/optimize-memory.sh                  |  16 -
 docs/src/development.adoc                          |  51 +--
 gradle.properties                                  |   4 +-
 scripts/cleanup-pods.sh                            |  12 -
 scripts/setup-minikube.sh                          |  30 --
 settings.gradle                                    |   1 -
 .../org/apache/cassandra/sidecar/models/Range.java |   3 +-
 .../org/apache/cassandra/sidecar/RangeTest.java    |  15 +-
 24 files changed, 79 insertions(+), 776 deletions(-)

diff --git a/README.md b/README.md
index 2bdf4ef..21991bf 100644
--- a/README.md
+++ b/README.md
@@ -9,7 +9,7 @@ Requirements
 ------------
   1. Java >= 1.8 (OpenJDK or Oracle), or Java 11
   2. Apache Cassandra 4.0.  We depend on virtual tables which is a 4.0 only feature.
-  3. A Kubernetes cluster for running integration tests.  [MiniKube](https://kubernetes.io/docs/tutorials/hello-minikube/) can be used to run Kubernetes locally.
+  3. [Docker](https://www.docker.com/products/docker-desktop/) for running integration tests.
 
 Getting started: Running The Sidecar
 --------------------------------------
@@ -25,46 +25,11 @@ Configuring Cassandra Instance
 While setting up cassandra instance, make sure the data directories of cassandra are in the path stored in sidecar.yaml file, else modify data directories path to point to the correct directories for stream APIs to work.
 
 Testing
----------
+-------
 
-We rely on Kubernetes for creating docker containers for integration tests.
+We rely on docker containers for integration tests.
 
-The easiest way to get started locally is by installing [Minikube](https://kubernetes.io/docs/tasks/tools/install-minikube/). 
-
-Start minikube with a command similar to the following.  Use a netmask appropriate for your local network, and allow minikube to use as much RAM as you can afford to:
-
-    minikube start --insecure-registry "192.168.0.0/16" --addons registry --memory 8G --cpus=4
-
-This will create a MiniKube cluster using the default driver.  On OSX, this is hyperkit.
-
-Enabling the tunnel is required in certain environments for tests to connect to the instances.
-
-In a separate tab (or background process) run the following:
-
-    minikube tunnel
-
-Check the dashboard to ensure your installation is working as expected:
-    
-    minikube dashboard
-
-Set the environment property for the Minikube container (we recommend you do this as part of your system profile):
-
-You can use an existing Kubernetes environment by setting the appropriate project properties either through environment variables
-
-    export SIDECAR_DOCKER_REGISTRY="http://$(minikube ip):5000"
-
-Gradle will register the required test containers with the local docker registry.    You can enable this after setting up Minikube by doing the following:
-
-*Note*: If using MiniKube, the Docker daemon will need to be configured to push to your Minikube repo insecurely.  
-This should be added to the `daemon.json` config, usually found in /etc/docker, or in the Docker Engine section of the docker preferences:
-
-      "insecure-registries": [
-        "192.168.64.14:5000"
-      ]
-    
-You can use `build`, `test` to build & test the project.
-
-Please see the developer documentation in docs/src/development.adoc for more information.
+The only requirement is to install and run [Docker](https://www.docker.com/products/docker-desktop/) on your test machine.
 
 CircleCI Testing
 -----------------
diff --git a/build.gradle b/build.gradle
index dc402ac..87cc251 100644
--- a/build.gradle
+++ b/build.gradle
@@ -31,52 +31,8 @@ allprojects {
     apply plugin: 'checkstyle'
     apply plugin: "com.github.spotbugs"
 
-    tasks.register("configureKubernetes") {
-        // figure out the docker registry, make this as easy as possible for new folks
-        def dockerRegistryString = System.getenv("SIDECAR_DOCKER_REGISTRY")
-        if(dockerRegistryString != null) {
-            logger.info("Using ENV SIDECAR_DOCKER_REGISTRY ${dockerRegistryString}")
-        }
-        else {
-            // SIDECAR_DOCKER_REGISTRY not set (likely a local environment), let's try minikube
-            logger.info("Looking for minikube ip")
-            new ByteArrayOutputStream().withStream { os ->
-                try {
-
-                def result = exec {
-                    executable = "minikube"
-                    args = ["ip"]
-                    standardOutput = os
-                }
-                def output = os.toString()
-                if(output != "") {
-                    dockerRegistryString = "http://${output.trim()}:5000" // minikube uses port 5000
-                    logger.info("Env variable SIDECAR_DOCKER_REGISTRY not defined, using output of minikube ip ${dockerRegistryString}")
-                } else {
-                    logger.warn("SIDECAR_DOCKER_REGISTRY not set and minikube ip failed")
-                }
-                if(result.exitValue != 0) {
-                    logger.error("Could not find minikube ip: exit code ${result.exitValue}")
-                }
-                }
-                catch (GradleException e) {
-                    // microk8s use this
-                    dockerRegistryString = "http://localhost:5000"
-                    logger.error("Could not auto configure docker registry, please set your SIDECAR_DOCKER_REGISTRY environment variable.  Using $dockerRegistryString")
-                }
-            }
-        }
-
-        ext.dockerTag = "latest"
-        ext.dockerGroup = System.getenv("SIDECAR_DOCKER_GROUP") ?: "cassandra_sidecar"
-        ext.kubernetesNamespace = System.getenv("SIDECAR_KUBERNETES_NAMESPACE") ?: "default"
-        ext.dockerRegistry = new URI(dockerRegistryString)
-        ext.dockerRegistryWithoutProtocol = dockerRegistry.getHost() + ":" + dockerRegistry.getPort()
-    }
-
     repositories {
         mavenCentral()
-        jcenter()
     }
 
     checkstyle {
@@ -173,7 +129,7 @@ dependencies {
 
     testImplementation "org.junit.jupiter:junit-jupiter-api:${project.junitVersion}"
     testImplementation "org.junit.jupiter:junit-jupiter-params:${project.junitVersion}"
-    testImplementation "org.assertj:assertj-core:3.14.0"
+    testImplementation "org.assertj:assertj-core:3.23.1"
     testRuntimeOnly "org.junit.jupiter:junit-jupiter-engine:${project.junitVersion}"
 
     testImplementation('org.cassandraunit:cassandra-unit-shaded:3.11.2.0')
@@ -191,7 +147,7 @@ dependencies {
     testFixturesImplementation('org.mockito:mockito-all:1.10.19')
     testFixturesImplementation("org.junit.jupiter:junit-jupiter-api:${project.junitVersion}")
     testFixturesImplementation("org.junit.jupiter:junit-jupiter-params:${project.junitVersion}")
-    testFixturesImplementation("org.assertj:assertj-core:3.14.0")
+    testFixturesImplementation("org.assertj:assertj-core:3.23.1")
     testFixturesImplementation("io.vertx:vertx-web:${project.vertxVersion}")
 }
 
diff --git a/cassandra-integration-tests/build.gradle b/cassandra-integration-tests/build.gradle
index 101bb1e..11c80a9 100644
--- a/cassandra-integration-tests/build.gradle
+++ b/cassandra-integration-tests/build.gradle
@@ -4,7 +4,6 @@ plugins {
 
 repositories {
     mavenCentral()
-    jcenter()
 }
 
 dependencies {
@@ -14,35 +13,14 @@ dependencies {
     implementation('ch.qos.logback:logback-classic:1.2.3')
     implementation('com.datastax.cassandra:cassandra-driver-core:3.9.0+')
 
-    testImplementation("io.kubernetes:client-java:${project.kubernetesClientVersion}")
-    testImplementation("io.kubernetes:client-java-extended:${project.kubernetesClientVersion}")
-
-    testImplementation("org.junit.jupiter:junit-jupiter-api:${project.junitVersion}")
-    testImplementation("org.junit.jupiter:junit-jupiter-params:${project.junitVersion}")
     testRuntimeOnly("org.junit.jupiter:junit-jupiter-engine:${project.junitVersion}")
-
-    testImplementation("org.assertj:assertj-core:3.16.0")
+    testImplementation('org.assertj:assertj-core:3.23.1')
+    testImplementation("org.testcontainers:testcontainers:${project.testcontainersVersion}")
+    testImplementation("org.testcontainers:cassandra:${project.testcontainersVersion}")
+    testImplementation("org.testcontainers:junit-jupiter:${project.testcontainersVersion}")
 }
 
-// we don't want to run the integration tests by default - they're heavy weight and won't run correctly
-// if kubernetes isn't present
-// we don't want the lack of local k8 to impede development
-// for the most part, the C* integrations should be simple, and non-integration tests should use mocks
+// Docker is required to run integration tests
 test {
-    useJUnitPlatform() {
-        excludeTags "integrationTest"
-    }
+    useJUnitPlatform()
 }
-
-tasks.register("integrationTest", Test) {
-    jacoco {
-        enabled = false
-    }
-    useJUnitPlatform() {
-        includeTags "integrationTest"
-    }
-    systemProperty "sidecar.dockerRegistry", configureKubernetes.ext.dockerRegistry.toString()
-    systemProperty "sidecar.kubernetesNamespace", configureKubernetes.ext.kubernetesNamespace.toString()
-    systemProperty "sidecar.dockerGroup", configureKubernetes.ext.dockerGroup.toString()
-    group = "verification"
-}
\ No newline at end of file
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/DelegateTest.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java
similarity index 68%
rename from cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/DelegateTest.java
rename to cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java
index be775cd..0d9f237 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/DelegateTest.java
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java
@@ -16,21 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.sidecar.common.testing;
+package org.apache.cassandra.sidecar.common;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
-import io.kubernetes.client.openapi.ApiException;
-import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
-import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
-import org.apache.cassandra.sidecar.common.SimpleCassandraVersion;
+import com.datastax.driver.core.NettyOptions;
+import org.apache.cassandra.sidecar.common.testing.CassandraIntegrationTest;
+import org.apache.cassandra.sidecar.common.testing.CassandraTestContext;
 import org.apache.cassandra.sidecar.mocks.V30;
+
 import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * Insures the Delegate works correctly
  */
-public class DelegateTest
+class DelegateTest
 {
     @CassandraIntegrationTest
     void testCorrectVersionIsEnabled(CassandraTestContext context)
@@ -43,7 +44,7 @@ public class DelegateTest
     }
 
     @CassandraIntegrationTest
-    void testHealthCheck(CassandraTestContext context) throws InterruptedException, ApiException, IOException
+    void testHealthCheck(CassandraTestContext context) throws IOException, InterruptedException
     {
         CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
         CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session);
@@ -51,17 +52,18 @@ public class DelegateTest
         delegate.checkSession();
         delegate.healthCheck();
 
-        assertThat(delegate.isUp()).isTrue();
+        assertThat(delegate.isUp()).as("health check succeeds").isTrue();
 
-        context.container.disableBinary();
+        context.container.execInContainer("nodetool", "disablebinary");
 
         delegate.healthCheck();
-        assertThat(delegate.isUp()).isFalse();
+        assertThat(delegate.isUp()).as("health check fails after binary has been disabled").isFalse();
 
-        context.container.enableBinary();
+        context.container.execInContainer("nodetool", "enablebinary");
 
+        TimeUnit.SECONDS.sleep(1);
         delegate.healthCheck();
 
-        assertThat(delegate.isUp()).isTrue();
+        assertThat(delegate.isUp()).as("health check succeeds after binary has been enabled").isTrue();
     }
 }
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/StatusTest.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/StatusTest.java
index cf27d8d..b53f689 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/StatusTest.java
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/StatusTest.java
@@ -17,27 +17,31 @@
  */
 
 package org.apache.cassandra.sidecar.common;
+
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.DisplayName;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.datastax.driver.core.ResultSet;
 import com.datastax.driver.core.Session;
 import org.apache.cassandra.sidecar.common.testing.CassandraIntegrationTest;
 import org.apache.cassandra.sidecar.common.testing.CassandraTestContext;
 
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
 /**
  * Placeholder test
  */
-public class StatusTest
+class StatusTest
 {
     private static final Logger logger = LoggerFactory.getLogger(StatusTest.class);
 
     @BeforeEach
     void setupData(CassandraTestContext context)
     {
-        logger.info("setup {}", context.container);
+        assertThat(context.container.isRunning()).isTrue();
+        logger.info("Running Cassandra on host={}", context.container.getContactPoint());
     }
 
     @CassandraIntegrationTest
@@ -46,6 +50,7 @@ public class StatusTest
     {
         logger.info("test context in test {}", context);
         Session session = context.session.getLocalCql();
-        session.execute("SELECT * from system.peers_v2");
+        ResultSet rs = session.execute("SELECT * from system.peers_v2");
+        assertThat(rs).isNotNull();
     }
 }
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraPod.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraPod.java
deleted file mode 100644
index 65c6b76..0000000
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraPod.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * 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.testing;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import io.kubernetes.client.Exec;
-
-import io.kubernetes.client.openapi.ApiClient;
-import io.kubernetes.client.openapi.ApiException;
-import io.kubernetes.client.openapi.Configuration;
-
-import io.kubernetes.client.openapi.apis.CoreV1Api;
-import io.kubernetes.client.openapi.models.V1ContainerPort;
-import io.kubernetes.client.openapi.models.V1Pod;
-import io.kubernetes.client.openapi.models.V1PodBuilder;
-import io.kubernetes.client.openapi.models.V1Service;
-import io.kubernetes.client.openapi.models.V1ServiceBuilder;
-import io.kubernetes.client.openapi.models.V1ServicePort;
-import io.kubernetes.client.openapi.models.V1ServiceSpec;
-
-import io.kubernetes.client.util.ClientBuilder;
-import okhttp3.OkHttpClient;
-import okhttp3.Protocol;
-import okhttp3.internal.Util;
-
-/**
- * Manages a single instance of a Cassandra container
- */
-class CassandraPod
-{
-    private final URI dockerRegistry;
-    private final String image;
-    private final String namespace;
-    private final String dockerGroup;
-    private static final Logger logger = LoggerFactory.getLogger(CassandraPod.class);
-    private final String podName;
-
-    private final CoreV1Api coreV1Api;
-    private Boolean deleted = false;
-
-    String ip;
-    Integer port;
-
-    public CassandraPod(URI dockerRegistry, String dockerGroup, String image, String namespace, CoreV1Api coreV1Api)
-    {
-        this.dockerRegistry = dockerRegistry;
-        this.image = image;
-        this.namespace = namespace;
-        this.dockerGroup = dockerGroup;
-        this.podName = String.format("cassandra-%s", UUID.randomUUID());
-        this.coreV1Api = coreV1Api;
-    }
-
-    /**
-     * Creates a single pod using the system properties passed in through Gradle
-     * @param image
-     * @return
-     * @throws Exception
-     */
-    public static CassandraPod createFromProperties(String image) throws Exception
-    {
-        URI dockerRegistry = new URI(System.getProperty("sidecar.dockerRegistry"));
-        String namespace = System.getProperty("sidecar.kubernetesNamespace");
-        String dockerGroup = System.getProperty("sidecar.dockerGroup");
-
-        logger.info("Creating pod from registry {}, namespace {}, group {}", dockerRegistry, namespace, dockerGroup);
-
-        if (dockerRegistry == null)
-        {
-            throw new Exception("Docker registry required but sidecar.dockerRegistry = null");
-        }
-        if (namespace == null)
-        {
-            throw new Exception("sidecar.kubernetesNamespace is not defined and is required for K8 testing");
-        }
-
-        ApiClient apiClient = ClientBuilder.standard().build();
-
-        // this is a workaround for socket errors that show up in certain JVM versions...
-        // without it, the tests fail in CI.
-        // we can probably get rid of this when we either move to JDK 11 only or if the Kubernetes clienti s updated
-        OkHttpClient httpClient =
-                apiClient.getHttpClient().newBuilder()
-                        .protocols(Util.immutableList(Protocol.HTTP_1_1))
-                        .readTimeout(10, TimeUnit.SECONDS)
-                        .writeTimeout(10, TimeUnit.SECONDS)
-                        .connectTimeout(10, TimeUnit.SECONDS)
-                        .callTimeout(10, TimeUnit.SECONDS)
-                        .retryOnConnectionFailure(true)
-                        .build();
-        apiClient.setHttpClient(httpClient);
-
-        Configuration.setDefaultApiClient(apiClient);
-
-        logger.info("K8 client: {}", apiClient.getBasePath());
-
-        CoreV1Api coreV1Api = new CoreV1Api(apiClient);
-
-
-        return new CassandraPod(dockerRegistry, dockerGroup, image, namespace, coreV1Api);
-    }
-
-    public void start() throws ApiException, InterruptedException, CassandraPodException
-    {
-        // create a v1 deployment spec
-        String fullImage = getFullImageName();
-
-        // similar to the spec yaml file, just programmatic
-
-        HashMap<String, String> labels = getLabels();
-
-        V1Service serviceBuilder = getService();
-
-        logger.debug("Exposing service with: {}", serviceBuilder.toString());
-
-        try
-        {
-            coreV1Api.createNamespacedService(namespace, serviceBuilder, null, null, null);
-        }
-        catch (ApiException e)
-        {
-            logger.error("Unable to create namespaced service: {}", e.getMessage());
-            throw e;
-        }
-
-        // get the service
-        V1Service namespacedService = coreV1Api.readNamespacedService(podName, namespace, null, null, null);
-        logger.debug("Service result: {}", namespacedService);
-
-        V1ServiceSpec serviceSpec = namespacedService.getSpec();
-
-        logger.info("Starting container {}", fullImage);
-        V1Pod pod = getPod(fullImage, labels);
-
-        logger.debug("Pod spec: {}", pod);
-        V1Pod podResult = coreV1Api.createNamespacedPod(namespace, pod, null, null, null);
-        logger.debug("Pod result: {}", podResult);
-
-        int maxTime = 120;
-        V1Pod namespacedPod = null;
-        Boolean started = false;
-        String response = "";
-        int sleepTimeInMs = 1000;
-
-        for (int i = 0; i < maxTime; i++)
-        {
-            // we sleep in the beginning because the pod will never be ready right away
-            // sometimes K8 seems to hang in CI as well, so this might be enough to let the pod start
-            logger.debug("Reading namespace pod - sleeping for {}ms, ID: {}", sleepTimeInMs, podName);
-            try
-            {
-                Thread.sleep(sleepTimeInMs);
-            }
-            catch (InterruptedException e)
-            {
-                logger.error("Unable to sleep: {}", e.getMessage());
-                throw e;
-            }
-            namespacedPod = coreV1Api.readNamespacedPod(podName, namespace, null, null, null);
-            response = namespacedPod.getStatus().getPhase();
-            // not ready
-
-            if (!response.contentEquals("Running"))
-            {
-                continue;
-            }
-
-            started = namespacedPod.getStatus().getContainerStatuses().get(0).getStarted();
-            if (namespacedPod.getStatus().getContainerStatuses().get(0).getReady() && started)
-            {
-                logger.info("Pod startup OK");
-                break;
-            }
-
-            logger.info("Container not ready: {}", response);
-            Thread.sleep(1000);
-        }
-        if (!started)
-        {
-            throw new CassandraPodException("container not ready: " + response);
-        }
-        logger.debug("pod status: {}", namespacedPod);
-
-
-        ip = serviceSpec.getClusterIP();
-
-        List<V1ServicePort> ports = serviceSpec.getPorts();
-        port = ports.get(0).getPort();
-        logger.info("Cassandra pod {} running on {}:{}", podName, ip, port);
-    }
-
-    private HashMap<String, String> getLabels()
-    {
-        HashMap<String, String> labels = new HashMap<>();
-        labels.put("name", podName);
-        labels.put("purpose", "cassandra_sidecar_testing");
-        return labels;
-    }
-
-
-    private V1Service getService()
-    {
-        return new V1ServiceBuilder()
-                .withApiVersion("v1")
-                .withKind("Service")
-                .withNewMetadata()
-                .withName(podName)
-                .addToLabels("purpose", "cassandra_sidecar_testing")
-                .withNamespace(namespace)
-                .endMetadata()
-                .withNewSpec()
-                .withType("NodePort")
-                .addToPorts(new V1ServicePort().port(9042).protocol("TCP"))
-                .addToSelector("name", podName)
-                .endSpec()
-                .build();
-    }
-
-    private V1Pod getPod(String fullImage, HashMap<String, String> labels)
-    {
-        return new V1PodBuilder()
-                .withApiVersion("v1")
-                .withKind("Pod")
-                .withNewMetadata()
-                    .withName(podName)
-                    .withNamespace(namespace)
-                    .withLabels(labels)
-                    .endMetadata()
-                .withNewSpec()
-                    .addNewContainer()
-                        .withName(podName)
-                        .withImage(fullImage)
-                        .addToPorts(new V1ContainerPort().containerPort(9042))
-                        .withNewStartupProbe()
-                            .withNewTcpSocket()
-                                .withNewPort(9042)
-                                .endTcpSocket()
-                            .withInitialDelaySeconds(5)
-                            .withPeriodSeconds(3)
-                            .withFailureThreshold(30)
-                            .endStartupProbe()
-                        .endContainer()
-                    .endSpec()
-                .build();
-    }
-
-    public String getFullImageName()
-    {
-        return String.format("%s:%d/%s/%s", dockerRegistry.getHost(), dockerRegistry.getPort(), dockerGroup, image);
-    }
-
-    public void disableBinary() throws InterruptedException, ApiException, IOException
-    {
-        nodetool(new String[] {"disablebinary"});
-    }
-
-    public void enableBinary() throws InterruptedException, ApiException, IOException
-    {
-        nodetool(new String[] {"enablebinary"});
-        // temporary sleep to ensure we start back up
-        Thread.sleep(5000);
-    }
-
-    /**
-     *
-     */
-    public void nodetool(String[] args) throws IOException, ApiException, InterruptedException
-    {
-        Exec exec = new Exec();
-        List<String> command = new ArrayList<>();
-        command.add("/cassandra/bin/nodetool");
-        Collections.addAll(command, args);
-        logger.info("Executing in container {}", command);
-
-        Process proc = exec.exec(namespace, podName, command.toArray(new String[0]), false, false);
-        proc.waitFor();
-        proc.destroy();
-    }
-
-
-    /**
-     * this is done asynchronously, so technically we could connect for a small window after the pod is deleted
-     * not recommended to use mid-test
-     */
-    public void delete()
-    {
-        if (deleted)
-        {
-            logger.info("Pod already deleted, skipping");
-            return;
-        }
-        deleteService();
-        deletePod();
-        deleted = true;
-
-    }
-    public String getIp()
-    {
-        return ip;
-    }
-
-    public Integer getPort()
-    {
-        return port;
-    }
-
-    private void deleteService()
-    {
-        try
-        {
-            logger.info("Deleting service {}", podName);
-            coreV1Api.deleteNamespacedService(podName, namespace, null, null, null, null, null, null);
-
-        }
-        catch (Exception ex)
-        {
-            logger.info(String.format("Could not delete service %s", podName), ex);
-        }
-    }
-
-    /**
-     * Tries to delete a pod.  Might fail.
-     * There's a variety of cases that can result in a pod not being deleted properly
-     */
-    private void deletePod()
-    {
-        try
-        {
-            logger.info("Deleting pod {}", podName);
-            coreV1Api.deleteNamespacedPod(podName, namespace, null, null, null, null, null, null);
-        }
-        catch (Exception ex)
-        {
-            logger.error(String.format("Unable to delete pod %s", podName), ex);
-        }
-    }
-}
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraPodException.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraPodException.java
deleted file mode 100644
index 6139a3e..0000000
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraPodException.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.testing;
-
-/**
- * Misc exception to be thrown when we don't know what's actually wrong
- */
-public class CassandraPodException extends Exception
-{
-    public CassandraPodException(String message)
-    {
-        super(message);
-    }
-}
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
index eb4ef65..eda88bb 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.sidecar.common.testing;
 import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.ICassandraAdapter;
 import org.apache.cassandra.sidecar.common.SimpleCassandraVersion;
+import org.testcontainers.containers.CassandraContainer;
 
 /**
  * Passed to integration tests.
@@ -31,10 +32,10 @@ public class CassandraTestContext
 {
     public final CQLSession session;
     public final SimpleCassandraVersion version;
-    public final CassandraPod container;
+    public final CassandraContainer<?> container;
     public final ICassandraAdapter cassandra;
 
-    CassandraTestContext(SimpleCassandraVersion version, CassandraPod container, CQLSession session,
+    CassandraTestContext(SimpleCassandraVersion version, CassandraContainer<?> container, CQLSession session,
                          ICassandraAdapter cassandra)
     {
         this.version = version;
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
index e8a38a9..515db8e 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
@@ -34,10 +34,12 @@ import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.datastax.driver.core.NettyOptions;
 import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.ICassandraAdapter;
 import org.apache.cassandra.sidecar.common.ICassandraFactory;
 import org.apache.cassandra.sidecar.common.SimpleCassandraVersion;
+import org.testcontainers.containers.CassandraContainer;
 
 /**
  * Creates a test per version of Cassandra we are testing
@@ -109,14 +111,15 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
                     @Override
                     public void beforeEach(ExtensionContext context) throws Exception
                     {
-                        // spin up a C* instance using Kubernetes
+                        // spin up a C* instance using Testcontainers
                         ICassandraFactory factory = version.getFactory();
 
-                        CassandraPod container = CassandraPod.createFromProperties(version.getImage());
+                        CassandraContainer<?> container = new CassandraContainer<>(version.getImage())
+                                                          .withExposedPorts(9042);
                         container.start();
                         logger.info("Testing {} against docker container", version);
 
-                        CQLSession session = new CQLSession(container.getIp(), container.getPort(), 5000);
+                        CQLSession session = new CQLSession(container.getContactPoint(), new NettyOptions());
 
                         SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(version.getVersion());
 
@@ -134,14 +137,9 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
              */
             private AfterTestExecutionCallback postProcessor()
             {
-                return new AfterTestExecutionCallback()
-                {
-                    @Override
-                    public void afterTestExecution(ExtensionContext context) throws Exception
-                    {
-                        // tear down the docker instance
-                        cassandraTestContext.container.delete();
-                    }
+                return context1 -> {
+                    // tear down the docker instance
+                    cassandraTestContext.container.stop();
                 };
             }
 
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
index 233c9a8..4703f92 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
@@ -28,17 +28,17 @@ import org.apache.cassandra.sidecar.cassandra40.Cassandra40Factory;
  * versions of Cassandra.  This may be due to releases that don't add new features that would affect the sidecar,
  * but we still want to test those versions specifically to avoid the chance of regressions.
  *
- * At the moment, it's returning a hard coded list.  We could / should probably load this from a configuration and make
+ * <p>At the moment, it's returning a hard coded list.  We could / should probably load this from a configuration and make
  * it possible to override it, so teams that customize C* can run and test their own implementation
  *
- * Ideally, we'd probably have concurrent runs of the test infrastructure each running tests against one specific
+ * <p>Ideally, we'd probably have concurrent runs of the test infrastructure each running tests against one specific
  * version of C*, but we don't need that yet given we only have one version.
  */
 public class TestVersionSupplier
 {
     Stream<TestVersion> getTestVersions()
     {
-        return Stream.of(new TestVersion("4.0.0", new Cassandra40Factory(), "cassandra40"));
+        return Stream.of(new TestVersion("4.0.0", new Cassandra40Factory(), "cassandra:4.0.6"));
     }
 
 }
diff --git a/cassandra-integration-tests/src/test/resources/logback-test.xml b/cassandra-integration-tests/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..cef0461
--- /dev/null
+++ b/cassandra-integration-tests/src/test/resources/logback-test.xml
@@ -0,0 +1,12 @@
+<configuration>
+
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} %-5level %logger - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="INFO">
+        <appender-ref ref="STDOUT"/>
+    </root>
+</configuration>
diff --git a/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40Factory.java b/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40Factory.java
index 93e5aa6..d742949 100644
--- a/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40Factory.java
+++ b/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40Factory.java
@@ -18,12 +18,10 @@
 
 package org.apache.cassandra.sidecar.cassandra40;
 
-import java.util.List;
 import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.ICassandraAdapter;
 import org.apache.cassandra.sidecar.common.ICassandraFactory;
 import org.apache.cassandra.sidecar.common.MinimumVersion;
-import org.apache.cassandra.sidecar.common.NodeStatus;
 
 /**
  * Factory to produce the 4.0 adapter
@@ -34,12 +32,6 @@ public class Cassandra40Factory implements ICassandraFactory
     @Override
     public ICassandraAdapter create(CQLSession session)
     {
-        return new ICassandraAdapter()
-        {
-            public List<NodeStatus> getStatus()
-            {
-                return null;
-            }
-        };
+        return () -> null;
     }
 }
diff --git a/common/build.gradle b/common/build.gradle
index a4f50eb..04cf2f5 100644
--- a/common/build.gradle
+++ b/common/build.gradle
@@ -34,7 +34,7 @@ dependencies {
 
     implementation('org.apache.commons:commons-lang3:3.12.0')
 
-    testImplementation("org.assertj:assertj-core:3.16.0")
+    testImplementation("org.assertj:assertj-core:3.23.1")
     testImplementation("org.junit.jupiter:junit-jupiter-api:${project.junitVersion}")
     testImplementation("org.junit.jupiter:junit-jupiter-params:${project.junitVersion}")
     testRuntimeOnly("org.junit.jupiter:junit-jupiter-engine:${project.junitVersion}")
diff --git a/containers/build.gradle b/containers/build.gradle
deleted file mode 100644
index 1215051..0000000
--- a/containers/build.gradle
+++ /dev/null
@@ -1,76 +0,0 @@
-import com.bmuschko.gradle.docker.tasks.image.*
-
-plugins {
-    id 'com.bmuschko.docker-remote-api'
-    id "de.undercouch.download" version "4.1.0"
-}
-
-docker {
-    registryCredentials {
-        url = "${-> configureKubernetes.ext.dockerRegistry.toString()}/v2/"
-    }
-}
-
-class BuildDockerConfig extends Copy {
-
-}
-
-tasks.register("download40", Download) {
-    src "https://downloads.apache.org/cassandra/${project.cassandra40Version}/apache-cassandra-${project.cassandra40Version}-bin.tar.gz"
-    dest "$buildDir/cassandra40/apache-cassandra-${project.cassandra40Version}-bin.tar.gz"
-    overwrite false
-}
-
-tasks.register("cassandra40", BuildDockerConfig) {
-    from("src/Cassandra40") {
-    }
-
-    from("src") {
-        include "docker-entrypoint.sh"
-        include "optimize-memory.sh"
-    }
-
-    into "$buildDir/cassandra40/"
-    dependsOn download40
-
-}
-
-tasks.register("generateDockerConfigs") {
-    doFirst {
-        mkdir "build"
-    }
-    dependsOn tasks.withType(BuildDockerConfig)
-}
-
-def getRemoteTag(name) {
-    return "${configureKubernetes.ext.dockerRegistryWithoutProtocol}/${configureKubernetes.ext.dockerGroup}/${name}:${configureKubernetes.ext.dockerTag}".toString()
-}
-
-tasks.register("buildImageCassandra40", DockerBuildImage) {
-    // the toString is required here, otherwise we get org.codehaus.groovy.runtime.GStringImpl cannot be cast to java.lang.String
-    def name = "cassandra40"
-    def localTag = "${name}:${configureKubernetes.ext.dockerTag}".toString()
-
-    // there might not be a group
-    def remoteTag = getRemoteTag(name)
-    // tag the private repo
-    tags = [localTag, remoteTag]
-    inputDir = file("$buildDir/${name}")
-    buildArgs = ["CASSANDRA_40_VERSION" : "$project.cassandra40Version"]
-    dependsOn "cassandra40"
-}
-
-tasks.register("publishCassandra40", DockerPushImage) {
-    def name = "cassandra40"
-    imageName = "${configureKubernetes.ext.dockerRegistryWithoutProtocol}/${configureKubernetes.ext.dockerGroup}/${name}"
-
-    dependsOn buildImageCassandra40
-}
-
-tasks.register("buildAll") {
-    dependsOn tasks.withType(DockerBuildImage)
-}
-tasks.register("pushAll") {
-    dependsOn tasks.withType(DockerPushImage)
-    dependsOn "buildAll"
-}
diff --git a/containers/src/Cassandra40/Dockerfile b/containers/src/Cassandra40/Dockerfile
deleted file mode 100644
index a0b1a04..0000000
--- a/containers/src/Cassandra40/Dockerfile
+++ /dev/null
@@ -1,19 +0,0 @@
-FROM ubuntu:bionic
-ARG CASSANDRA_40_VERSION
-RUN apt update && apt install -y openjdk-11-jdk-headless
-RUN mkdir /downloads
-WORKDIR /downloads/
-COPY apache-cassandra-$CASSANDRA_40_VERSION-bin.tar.gz .
-RUN tar zxvf apache-cassandra-$CASSANDRA_40_VERSION-bin.tar.gz
-RUN mv apache-cassandra-$CASSANDRA_40_VERSION /cassandra
-
-ADD docker-entrypoint.sh /cassandra/
-RUN chmod +x /cassandra/docker-entrypoint.sh
-
-ADD optimize-memory.sh .
-
-RUN chmod +x optimize-memory.sh && ./optimize-memory.sh
-
-EXPOSE 9042
-ENTRYPOINT ["/cassandra/docker-entrypoint.sh"]
-
diff --git a/containers/src/docker-entrypoint.sh b/containers/src/docker-entrypoint.sh
deleted file mode 100644
index ebf4e57..0000000
--- a/containers/src/docker-entrypoint.sh
+++ /dev/null
@@ -1,15 +0,0 @@
-#!/bin/bash
-set -e
-
-export CASSANDRA_ADDRESS=$(awk 'END{print $1}' /etc/hosts)
-
-# 4 tokens for faster startup (but still use tokens)
-sed -i -e "s/^\(num_tokens:\).*/\1 4 /g" /cassandra/conf/cassandra.yaml
-# default listens on localhost, can't connect to that
-sed -i -e "s/^\(rpc_address:\).*/\1 ${CASSANDRA_ADDRESS} /g" /cassandra/conf/cassandra.yaml
-
-#echo "-Xmx1G" >> /cassandra/conf/jvm-server.options
-#echo "-Xmn500M" >> /cassandra/conf/jvm-server.options
-
-cd /cassandra
-bin/cassandra -f -R
diff --git a/containers/src/optimize-memory.sh b/containers/src/optimize-memory.sh
deleted file mode 100644
index 999d5a0..0000000
--- a/containers/src/optimize-memory.sh
+++ /dev/null
@@ -1,16 +0,0 @@
-#!/usr/bin/env bash
-
-CONF_DIR=/cassandra/conf
-YAML=$CONF_DIR/cassandra.yaml
-
-
-sed -i 's/#MAX_HEAP_SIZE="4G"/MAX_HEAP_SIZE="512m"/' $CONF_DIR/cassandra-env.sh
-sed -i 's/#HEAP_NEWSIZE="800M"/HEAP_NEWSIZE="256m"/' $CONF_DIR/cassandra-env.sh
-sed -i 's/num_tokens: 256/num_tokens: 1/' $YAML
-echo 'phi_convict_threshold: 16' >> $YAML
-sed -i 's/concurrent_reads: 32/concurrent_reads: 4/' $YAML
-sed -i 's/concurrent_writes: 32/concurrent_writes: 4/' $YAML
-sed -i 's/concurrent_counter_writes: 32/concurrent_counter_writes: 4/' $YAML
-sed -i 's/# file_cache_size_in_mb: 512/file_cache_size_in_mb: 1/' $YAML
-
-rm -rf /cassandra/data/
\ No newline at end of file
diff --git a/docs/src/development.adoc b/docs/src/development.adoc
index 3f8b9a1..d89e68c 100644
--- a/docs/src/development.adoc
+++ b/docs/src/development.adoc
@@ -23,61 +23,12 @@ Implementation of ICassandraAdapter for Cassandra 4.0.
 
 ### Cassandra Integration Tests
 
-Cassandra integration tests leverage Kubernetes to create Cassandra nodes and test the different implementations
+Cassandra integration tests leverage Docker to create Cassandra nodes and test the different implementations
 of the ICassandraAdapters against real C* nodes.
 
 The integration tests will not run by default when running `./gradlew test` since they require a bit of setup and you may not have
 Kubernetes available.
 
-#### Mac Local Setup
-
-Running kubernetes locally on a Mac can be a bit tricky as it requires several steps.
-
-First, install minikube: https://kubernetes.io/docs/tasks/tools/install-minikube/
-
-There is a convenience script, located at `scripts/setup-minikube.sh`, that can be used to create the minikube environment and enable the registry for you.
-
-You will also need Docker: https://docs.docker.com/get-docker/ to build containers locally before pushing to the minikube registry.
-
-NOTE: If you have a registry already running you can skip the above step and instead set the SIDECAR_DOCKER_REGISTRY environment variable to the full URI of your registry..
-
-If you're using the local option, make sure you can push to an insecure registry.
-The following should be added to your docker configuration (under docker preferences -> docker engine).  Be sure to use whatever ip is reported by `minikube ip`:
-
-    {
-        "insecure-registries": ["192.168.64.16:5000"]
-    }
-
-You will need to run `minikube tunnel` to ensure your Mac can connect to the IP addresses inside the hyperkit virtual machine:
-
-    minikube tunnel
-
-Enter your password when prompted.  It'll redirect part of the 10.x address space to the hyperkit virtual machine.
-
-Once all this is completed, execute `./gradlew pushAll` to create all the containers required for integration testing.
-
-Sometimes minikube can run into issues starting up new jobs, especially if there are several running pods which failed to get cleaned up.  Running the following can help clean up the old pods:
-
-    scripts/cleanup-pods.sh
-
-#### Linux Setup
-
-The most straightforward setup on Linux is to use microk8s, as this relies on non virtualized containers, which give excellent performance.
-
-https://microk8s.io
-
-You will need to configure the SIDECAR_DOCKER_REGISTRY environment variable.  If you're using the built in microk8's registry you should configure the following:
-
-    export SIDECAR_DOCKER_REGISTRY="http://localhost:32000"
-
-NOTE: The MicroK8 project uses 32000 for its registry while minikube on MacOS uses port 5000.
-
-Please see the setup in `.circleci/setup-microk8.sh`.
-
-#### Push containers to the registry
-
-    ./gradlew pushAll
-
 #### Running Integration Tests
 
 Integration tests can be run with the following:
diff --git a/gradle.properties b/gradle.properties
index fa34b46..60341b3 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -1,5 +1,5 @@
 version=1.0-SNAPSHOT
-junitVersion=5.4.2
-kubernetesClientVersion=9.0.0
+junitVersion=5.9.1
+testcontainersVersion=1.17.5
 cassandra40Version=4.0.5
 vertxVersion=4.2.1
diff --git a/scripts/cleanup-pods.sh b/scripts/cleanup-pods.sh
deleted file mode 100755
index e046996..0000000
--- a/scripts/cleanup-pods.sh
+++ /dev/null
@@ -1,12 +0,0 @@
-#!/usr/bin/env bash
-
-echo "Existing pods and services"
-kubectl get pods
-kubectl get services
-
-echo "Cleaning up pods:"
-kubectl get pods | awk '{print $1}' | egrep '^cassandra-' | xargs -n 1 kubectl delete pod
-
-echo "Cleaning up services:"
-kubectl get services | awk '{print $1}' | egrep '^cassandra-' | xargs -n 1 kubectl delete service
-
diff --git a/scripts/setup-minikube.sh b/scripts/setup-minikube.sh
deleted file mode 100755
index 42ebb0e..0000000
--- a/scripts/setup-minikube.sh
+++ /dev/null
@@ -1,30 +0,0 @@
-#!/usr/bin/env bash
-
-# Script to start minikube locally and push the containers to the local registry
-# we need to use the hyperkit driver or we can't use the registry
-
-minikube start --insecure-registry "192.168.0.0/16" --insecure-registry "10.0.0.0/24"  --memory 8G --cpus=4 --vm=true
-minikube addons enable registry
-minikube addons enable dashboard
-minikube addons list
-
-kubectl get all
-
-echo "Be sure to configure your docker registry to allow for insecure image uploads:"
-
-echo ""
-echo "{"
-echo " \"insecure-registries\": [\"$(minikube ip):5000\"]  "
-echo "}"
-
-echo "Ensure your docker configuration (Docker preferences -> Docker Engine) allows for insecure pushes then press enter to publish the test containers."
-
-read var
-
-./gradlew buildAll pushAll
-
-echo "To allow the tests to connect to your containers, please run the following:"
-
-echo "minikube tunnel"
-
-echo "Remember to do this before running integration tests!"
\ No newline at end of file
diff --git a/settings.gradle b/settings.gradle
index c28cfe4..b71ceed 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -2,7 +2,6 @@ rootProject.name = "cassandra-sidecar"
 
 include "cassandra40"
 include "common"
-include "containers"
 include "docs"
 include "cassandra-integration-tests"
 
diff --git a/src/main/java/org/apache/cassandra/sidecar/models/Range.java b/src/main/java/org/apache/cassandra/sidecar/models/Range.java
index 4f8a006..e224a9c 100644
--- a/src/main/java/org/apache/cassandra/sidecar/models/Range.java
+++ b/src/main/java/org/apache/cassandra/sidecar/models/Range.java
@@ -116,7 +116,8 @@ public class Range
     private static IllegalArgumentException invalidRangeHeaderException(String rangeHeader)
     {
         return new IllegalArgumentException("Invalid range header: " + rangeHeader + ". " +
-                                            "Supported Range formats are bytes=<start>-<end>, bytes=<start>-, bytes=-<suffix-length>");
+                                            "Supported Range formats are bytes=<start>-<end>, bytes=<start>-, " +
+                                            "bytes=-<suffix-length>");
     }
 
     // An initialized range is always valid; invalid params fail range initialization.
diff --git a/src/test/java/org/apache/cassandra/sidecar/RangeTest.java b/src/test/java/org/apache/cassandra/sidecar/RangeTest.java
index 2d301ff..baa87b6 100644
--- a/src/test/java/org/apache/cassandra/sidecar/RangeTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/RangeTest.java
@@ -71,7 +71,8 @@ public class RangeTest
         {
             Range.parseHeader(rangeHeader, Long.MAX_VALUE);
         });
-        String msg = "Invalid range header: bytes=2344--3432. Supported Range formats are bytes=<start>-<end>, bytes=<start>-, bytes=-<suffix-length>";
+        String msg = "Invalid range header: bytes=2344--3432. Supported Range formats are bytes=<start>-<end>, " +
+                     "bytes=<start>-, bytes=-<suffix-length>";
         assertEquals(msg, thrownException.getMessage());
     }
 
@@ -107,7 +108,8 @@ public class RangeTest
         {
             Range.parseHeader(rangeVal, 5);
         });
-        String msg = "Invalid range header: bits=0-. Supported Range formats are bytes=<start>-<end>, bytes=<start>-, bytes=-<suffix-length>";
+        String msg = "Invalid range header: bits=0-. Supported Range formats are bytes=<start>-<end>, " +
+                     "bytes=<start>-, bytes=-<suffix-length>";
         assertEquals(msg, thrownException.getMessage());
     }
 
@@ -128,12 +130,14 @@ public class RangeTest
         {
             Range.parseHeader(rangeHeader, Long.MAX_VALUE);
         });
-        String msg = "Invalid range header: bytes=0-19223372036854775807. Supported Range formats are bytes=<start>-<end>, bytes=<start>-, bytes=-<suffix-length>";
+        String msg = "Invalid range header: bytes=0-19223372036854775807. Supported Range formats are " +
+                     "bytes=<start>-<end>, bytes=<start>-, bytes=-<suffix-length>";
         assertEquals(msg, thrownException.getMessage());
     }
 
     @Test
-    public void testIntersect() {
+    public void testIntersect()
+    {
         Range range1, range2, expected;
         range1 = Range.of(5, 10);
         range2 = Range.of(9, 15);
@@ -156,7 +160,8 @@ public class RangeTest
     }
 
     @Test
-    public void testRangesDoNotIntersect() {
+    public void testRangesDoNotIntersect()
+    {
         Range range1 = Range.of(1, 5);
         Range range2 = Range.of(9, 15);
 


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