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/11/18 18:16:30 UTC

[cassandra-sidecar] branch trunk updated: CASSANDRASC-47: Introduce JMX foundation in Sidecar

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 02cc654  CASSANDRASC-47: Introduce JMX foundation in Sidecar
02cc654 is described below

commit 02cc6548f291528e9749a51d103463f9552f4b4e
Author: Francisco Guerrero <fr...@apple.com>
AuthorDate: Wed Nov 2 12:54:54 2022 -0700

    CASSANDRASC-47: Introduce JMX foundation in Sidecar
    
    In this commit, we introduce the JMX foundation in Sidecar to enable the ability to communicate
    with the Cassandra process. This commit adds new configuration parameters to configure the
    a `org.apache.cassandra.sidecar.common.JmxClient`. This client is available as part of the Cassandra
    delegate.
    
    A new interface is introduced and exposed through the `org.apache.cassandra.sidecar.common.ICassandraAdapter`.
    The new interface `StorageOperations` is intended to interface with the Cassandra StorageService.
    This commit provides an example implementation of the `takeSnapshot` method, which is also found
    in the Cassandra code base. This should allow us to interact with the Cassandra process to create
    snapshots.
    
    A fix is required in the `CassandraSidecarDaemon` class, where the `healthCheck` runs the first time
    only after the configured health check frequency (millis) has passed. This causes issues in the unit
    tests as well as the actual execution of the service, as the `adapter` will be `null` until the first
    health check is performed. To fix this issue, we perform a health check right after the server
    successfully starts up.
    
    Additional integration tests are added for testing the JMX integration with Cassandra. In the test,
    we spin up a new Cassandra container using `testcontainers` and we perform validation against the
    StorageService in Cassandra.
    
    Co-authored-by: Doug Rohrer <dr...@apple.com>
    
    patch by Francisco Guerrero, Doug Rohrer; reviewed by Yifan Cai, Dinesh Joshi for CASSANDRASC-47
---
 CHANGES.txt                                        |   5 +
 build.gradle                                       |   8 +-
 cassandra-integration-tests/build.gradle           |  15 +-
 .../cassandra/sidecar/common/DelegateTest.java     |   7 +-
 .../cassandra/sidecar/common/JmxClientTest.java    |  58 +++++
 .../common/containers/CassandraContainer.java      |  50 ++++
 .../common/testing/CassandraIntegrationTest.java   |   0
 .../common/testing/CassandraTestContext.java       |  23 +-
 .../common/testing/CassandraTestTemplate.java      |  79 ++++---
 .../sidecar/common/testing/TestVersion.java        |   0
 .../common/testing/TestVersionSupplier.java        |   4 +-
 cassandra40/build.gradle                           |   1 +
 .../sidecar/cassandra40/Cassandra40Factory.java    |  30 ++-
 .../cassandra40/Cassandra40StorageOperations.java  |  61 +++++
 .../sidecar/cassandra40/StorageJmxOperations.java  |  18 ++
 .../cassandra/sidecar/common/CQLSession.java       |  20 +-
 .../sidecar/common/CassandraAdapterDelegate.java   |  14 +-
 .../sidecar/common/ICassandraAdapter.java          |   5 +
 .../sidecar/common/ICassandraFactory.java          |   9 +-
 .../apache/cassandra/sidecar/common/JmxClient.java | 209 +++++++++++++++++
 .../sidecar/common/MockCassandraFactory.java       |   2 +-
 ...assandraAdapter.java => StorageOperations.java} |  24 +-
 .../org/apache/cassandra/sidecar/mocks/V30.java    |   3 +-
 .../org/apache/cassandra/sidecar/mocks/V40.java    |   3 +-
 .../org/apache/cassandra/sidecar/mocks/V41.java    |   3 +-
 .../cassandra/sidecar/common/JmxClientTest.java    | 251 ++++++++++++++++++++
 .../src/test/resources/testJmxPassword.properties  |   1 +
 spotbugs-exclude.xml                               |  11 +-
 src/main/dist/conf/sidecar.yaml                    |  10 +
 .../cassandra/sidecar/CassandraSidecarDaemon.java  |  44 +++-
 .../apache/cassandra/sidecar/Configuration.java    |  23 +-
 .../org/apache/cassandra/sidecar/MainModule.java   | 144 +-----------
 .../sidecar/YAMLSidecarConfiguration.java          | 261 +++++++++++++++++++++
 .../sidecar/cluster/InstancesConfigImpl.java       |   6 +
 .../sidecar/cluster/instance/InstanceMetadata.java |  18 +-
 .../cluster/instance/InstanceMetadataImpl.java     |  34 ++-
 .../cassandra/sidecar/utils/YAMLKeyConstants.java  |   7 +
 .../cassandra/sidecar/ConfigurationTest.java       | 118 ----------
 .../sidecar/YAMLSidecarConfigurationTest.java      |  84 +++++++
 .../cassandra/sidecar/snapshots/SnapshotUtils.java |  21 +-
 src/test/resources/sidecar_multiple_instances.yaml |  12 +-
 src/test/resources/sidecar_single_instance.yaml    |   5 +
 .../sidecar_with_single_multiple_instances.yaml    |  17 +-
 43 files changed, 1330 insertions(+), 388 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 1c92ca5..dd7f1e1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,10 @@
 1.0.0
 -----
+ * Introduce JMX foundation in Sidecar (CASSANDRASC-47)
+ * Delegate methods to the RateLimiter (CASSANDRASC-45)
+ * Add Schema API (CASSANDRASC-43)
+ * Refactor health check to use vertx timer (CASSANDRASC-44)
+ * Migrate minikube to testcontainers for integration tests (CASSANDRASC-46)
  * Refactor range and add toString (CASSANDRASC-41)
  * Fix search in list snapshot endpoint (CASSANDRASC-40)
  * Allow Cassandra input validation to be configurable (CASSANDRAC-39)
diff --git a/build.gradle b/build.gradle
index baa6168..c81c4c4 100644
--- a/build.gradle
+++ b/build.gradle
@@ -88,10 +88,7 @@ sourceSets {
         java {
             compileClasspath += main.output + test.output
             runtimeClasspath += main.output + test.output
-            srcDir file('src/integration/java')
-        }
-        resources {
-            srcDirs = [main.resources, "src/integration/resources"]
+            srcDir file('src/test/integration')
         }
     }
 }
@@ -99,8 +96,7 @@ sourceSets {
 configurations {
     jolokia
 
-    integrationTestCompile.extendsFrom testCompile
-    integrationTestRuntime.extendsFrom testRuntime
+    integrationTestImplementation.extendsFrom testImplementation
 }
 
 dependencies {
diff --git a/cassandra-integration-tests/build.gradle b/cassandra-integration-tests/build.gradle
index 54068ba..634742e 100644
--- a/cassandra-integration-tests/build.gradle
+++ b/cassandra-integration-tests/build.gradle
@@ -1,5 +1,6 @@
 plugins {
     id 'java'
+    id "java-test-fixtures"
 }
 
 repositories {
@@ -7,22 +8,24 @@ repositories {
 }
 
 dependencies {
-    testImplementation project(":cassandra40")
+    testFixturesApi project(":cassandra40")
 
     implementation('ch.qos.logback:logback-core:1.2.3')
     implementation('ch.qos.logback:logback-classic:1.2.3')
     implementation('com.datastax.cassandra:cassandra-driver-core:3.9.0+')
 
     testRuntimeOnly("org.junit.jupiter:junit-jupiter-engine:${project.junitVersion}")
-    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}")
+    testFixturesApi('org.assertj:assertj-core:3.23.1')
+    testFixturesApi("org.testcontainers:testcontainers:${project.testcontainersVersion}")
+    testFixturesApi("org.testcontainers:cassandra:${project.testcontainersVersion}")
+    testFixturesApi("org.testcontainers:junit-jupiter:${project.testcontainersVersion}")
 }
 
 // Docker is required to run integration tests
 test {
-    useJUnitPlatform()
+    useJUnitPlatform() {
+        excludeTags "integrationTest"
+    }
 }
 
 tasks.register("integrationTest", Test) {
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java
index 0d9f237..a0b15dc 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/DelegateTest.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.sidecar.common;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
-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;
@@ -29,7 +28,7 @@ import org.apache.cassandra.sidecar.mocks.V30;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /**
- * Insures the Delegate works correctly
+ * Ensures the Delegate works correctly
  */
 class DelegateTest
 {
@@ -37,7 +36,7 @@ class DelegateTest
     void testCorrectVersionIsEnabled(CassandraTestContext context)
     {
         CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
-        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session);
+        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session, context.jmxClient);
         delegate.checkSession();
         SimpleCassandraVersion version = delegate.getVersion();
         assertThat(version).isNotNull();
@@ -47,7 +46,7 @@ class DelegateTest
     void testHealthCheck(CassandraTestContext context) throws IOException, InterruptedException
     {
         CassandraVersionProvider provider = new CassandraVersionProvider.Builder().add(new V30()).build();
-        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session);
+        CassandraAdapterDelegate delegate = new CassandraAdapterDelegate(provider, context.session, context.jmxClient);
 
         delegate.checkSession();
         delegate.healthCheck();
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java
new file mode 100644
index 0000000..a065136
--- /dev/null
+++ b/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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;
+
+import org.apache.cassandra.sidecar.common.testing.CassandraIntegrationTest;
+import org.apache.cassandra.sidecar.common.testing.CassandraTestContext;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test to ensure connectivity with the JMX client
+ */
+public class JmxClientTest
+{
+    private static final String SS_OBJ_NAME = "org.apache.cassandra.db:type=StorageService";
+
+    @CassandraIntegrationTest
+    void testJmxConnectivity(CassandraTestContext context)
+    {
+        String opMode = context.jmxClient.proxy(SSProxy.class, SS_OBJ_NAME)
+                                         .getOperationMode();
+        assertThat(opMode).isNotNull();
+        assertThat(opMode).isIn("LEAVING", "JOINING", "NORMAL", "DECOMMISSIONED", "CLIENT");
+    }
+
+    @CassandraIntegrationTest
+    void testConsumerCall(CassandraTestContext context)
+    {
+        context.jmxClient.proxy(SSProxy.class, SS_OBJ_NAME)
+                         .refreshSizeEstimates();
+    }
+
+    /**
+     * An interface that pulls a method from the Cassandra Storage Service Proxy
+     */
+    public interface SSProxy
+    {
+        String getOperationMode();
+
+        void refreshSizeEstimates();
+    }
+}
diff --git a/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/containers/CassandraContainer.java b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/containers/CassandraContainer.java
new file mode 100644
index 0000000..9b6bb07
--- /dev/null
+++ b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/containers/CassandraContainer.java
@@ -0,0 +1,50 @@
+/*
+ * 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.containers;
+
+/**
+ * A Cassandra container that adds functionality to the testcontainers
+ * {@link org.testcontainers.containers.CassandraContainer} for JMX.
+ */
+public class CassandraContainer extends org.testcontainers.containers.CassandraContainer<CassandraContainer>
+{
+    public static final String RMI_SERVER_HOSTNAME = "127.0.0.1";
+    public static final Integer JMX_PORT = 7199;
+
+    /**
+     * Constructs a new {@link CassandraContainer} with the {@link #JMX_PORT JMX port} exposed
+     *
+     * @param dockerImageName the name of the docker image to use for the container
+     */
+    public CassandraContainer(String dockerImageName)
+    {
+        super(dockerImageName);
+        addFixedExposedPort(JMX_PORT, JMX_PORT);
+
+        addEnv("LOCAL_JMX", "no");
+        addEnv("JVM_EXTRA_OPTS", "-Dcom.sun.management.jmxremote=true"
+                                 + " -Djava.rmi.server.hostname=" + RMI_SERVER_HOSTNAME
+                                 + " -Dcom.sun.management.jmxremote.port=" + JMX_PORT
+                                 + " -Dcom.sun.management.jmxremote.rmi.port=" + JMX_PORT
+                                 + " -Dcom.sun.management.jmxremote.authenticate=false"
+                                 + " -Dcom.sun.management.jmxremote.local.only=false"
+                                 + " -Dcom.sun.management.jmxremote.ssl=false"
+        );
+    }
+}
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraIntegrationTest.java b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraIntegrationTest.java
similarity index 100%
rename from cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraIntegrationTest.java
rename to cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraIntegrationTest.java
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
similarity index 71%
rename from cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
rename to cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
index eda88bb..79020b5 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
+++ b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraTestContext.java
@@ -20,8 +20,9 @@ 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.JmxClient;
 import org.apache.cassandra.sidecar.common.SimpleCassandraVersion;
-import org.testcontainers.containers.CassandraContainer;
+import org.apache.cassandra.sidecar.common.containers.CassandraContainer;
 
 /**
  * Passed to integration tests.
@@ -30,28 +31,34 @@ import org.testcontainers.containers.CassandraContainer;
  */
 public class CassandraTestContext
 {
+    public final JmxClient jmxClient;
     public final CQLSession session;
     public final SimpleCassandraVersion version;
-    public final CassandraContainer<?> container;
+    public final CassandraContainer container;
     public final ICassandraAdapter cassandra;
 
-    CassandraTestContext(SimpleCassandraVersion version, CassandraContainer<?> container, CQLSession session,
+    CassandraTestContext(SimpleCassandraVersion version,
+                         CassandraContainer container,
+                         CQLSession session,
+                         JmxClient jmxClient,
                          ICassandraAdapter cassandra)
     {
         this.version = version;
         this.container = container;
         this.session = session;
         this.cassandra = cassandra;
+        this.jmxClient = jmxClient;
     }
 
     @Override
     public String toString()
     {
         return "CassandraTestContext{" +
-                "session=" + session +
-                ", version=" + version +
-                ", container=" + container +
-                ", cassandra=" + cassandra +
-                '}';
+               "jmxClient=" + jmxClient +
+               ", session=" + session +
+               ", version=" + version +
+               ", container=" + container +
+               ", cassandra=" + cassandra +
+               '}';
     }
 }
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
similarity index 61%
rename from cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
rename to cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
index 515db8e..2165c2c 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
+++ b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/CassandraTestTemplate.java
@@ -30,7 +30,6 @@ import org.junit.jupiter.api.extension.ParameterContext;
 import org.junit.jupiter.api.extension.ParameterResolver;
 import org.junit.jupiter.api.extension.TestTemplateInvocationContext;
 import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,18 +37,22 @@ 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.JmxClient;
 import org.apache.cassandra.sidecar.common.SimpleCassandraVersion;
-import org.testcontainers.containers.CassandraContainer;
+import org.apache.cassandra.sidecar.common.containers.CassandraContainer;
+
+import static org.apache.cassandra.sidecar.common.containers.CassandraContainer.JMX_PORT;
+import static org.apache.cassandra.sidecar.common.containers.CassandraContainer.RMI_SERVER_HOSTNAME;
 
 /**
  * Creates a test per version of Cassandra we are testing
  * Tests must be marked with {@link CassandraIntegrationTest}
- *
- *  This is a mix of parameterized tests + a custom extension.  we need to be able to provide the test context
- *  to each test (like an extension) but also need to create multiple tests (like parameterized tests).  Unfortunately
- *  the two don't play well with each other.  You can't get access to the parameters from the extension.
- *  This test template allows us full control of the test lifecycle and lets us tightly couple the context to each test
- *  we generate, since the same test can be run for multiple versions of C*.
+ * <p>
+ * This is a mix of parameterized tests + a custom extension.  we need to be able to provide the test context
+ * to each test (like an extension) but also need to create multiple tests (like parameterized tests).  Unfortunately
+ * the two don't play well with each other.  You can't get access to the parameters from the extension.
+ * This test template allows us full control of the test lifecycle and lets us tightly couple the context to each test
+ * we generate, since the same test can be run for multiple versions of C*.
  */
 public class CassandraTestTemplate implements TestTemplateInvocationContextProvider
 {
@@ -67,14 +70,16 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
     public Stream<TestTemplateInvocationContext> provideTestTemplateInvocationContexts(ExtensionContext context)
     {
         return new TestVersionSupplier().getTestVersions()
-                .map(v -> invocationContext(v, context));
+                                        .map(v -> invocationContext(v, context));
     }
 
     /**
+     * Returns a {@link TestTemplateInvocationContext}
      *
-     * @param version
-     * @param context
-     * @return
+     * @param version a version for the test
+     * @param context the <em>context</em> in which the current test or container is being executed.
+     * @return the <em>context</em> of a single invocation of a
+     *         {@linkplain org.junit.jupiter.api.TestTemplate test template}
      */
     private TestTemplateInvocationContext invocationContext(TestVersion version, ExtensionContext context)
     {
@@ -85,8 +90,8 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
             /**
              * A display name can be configured per test still - this adds the C* version we're testing automatically
              * as a suffix to the name
-             * @param invocationIndex
-             * @return
+             * @param invocationIndex the index to the invocation
+             * @return the display name
              */
             @Override
             public String getDisplayName(int invocationIndex)
@@ -96,7 +101,7 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
 
             /**
              * Used to register the extensions required to start and stop the docker environment
-             * @return
+             * @return a list of registered {@link Extension extensions}
              */
             @Override
             public List<Extension> getAdditionalExtensions()
@@ -106,38 +111,37 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
 
             private BeforeEachCallback beforeEach()
             {
-                return new BeforeEachCallback()
-                {
-                    @Override
-                    public void beforeEach(ExtensionContext context) throws Exception
-                    {
-                        // spin up a C* instance using Testcontainers
-                        ICassandraFactory factory = version.getFactory();
+                return beforeEachCtx -> {
+                    // spin up a C* instance using Testcontainers
+                    ICassandraFactory factory = version.getFactory();
 
-                        CassandraContainer<?> container = new CassandraContainer<>(version.getImage())
-                                                          .withExposedPorts(9042);
-                        container.start();
-                        logger.info("Testing {} against docker container", version);
+                    CassandraContainer container = new CassandraContainer(version.getImage());
+                    container.start();
+                    logger.info("Testing {} against docker container", version);
 
-                        CQLSession session = new CQLSession(container.getContactPoint(), new NettyOptions());
+                    CQLSession session = new CQLSession(container.getContactPoint(), new NettyOptions());
+                    JmxClient jmxClient = new JmxClient(RMI_SERVER_HOSTNAME, JMX_PORT);
 
-                        SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(version.getVersion());
+                    SimpleCassandraVersion versionParsed = SimpleCassandraVersion.create(version.getVersion());
 
-                        ICassandraAdapter cassandra = factory.create(session);
+                    ICassandraAdapter cassandra = factory.create(session, jmxClient);
 
-                        cassandraTestContext = new CassandraTestContext(versionParsed, container, session, cassandra);
-                        logger.info("Created test context {}", cassandraTestContext);
-                    }
+                    cassandraTestContext = new CassandraTestContext(versionParsed,
+                                                                    container,
+                                                                    session,
+                                                                    jmxClient,
+                                                                    cassandra);
+                    logger.info("Created test context {}", cassandraTestContext);
                 };
             }
 
             /**
              * Shuts down the docker container when the test is finished
-             * @return
+             * @return the {@link AfterTestExecutionCallback}
              */
             private AfterTestExecutionCallback postProcessor()
             {
-                return context1 -> {
+                return postProcessorCtx -> {
                     // tear down the docker instance
                     cassandraTestContext.container.stop();
                 };
@@ -145,7 +149,7 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
 
             /**
              * Required for Junit to know the CassandraTestContext can be used in these tests
-             * @return
+             * @return a {@link ParameterResolver}
              */
             private ParameterResolver parameterResolver()
             {
@@ -156,8 +160,8 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
                                                      ExtensionContext extensionContext)
                     {
                         return parameterContext.getParameter()
-                                .getType()
-                                .equals(CassandraTestContext.class);
+                                               .getType()
+                                               .equals(CassandraTestContext.class);
                     }
 
                     @Override
@@ -168,7 +172,6 @@ public class CassandraTestTemplate implements TestTemplateInvocationContextProvi
                     }
                 };
             }
-
         };
     }
 }
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersion.java b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/TestVersion.java
similarity index 100%
rename from cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersion.java
rename to cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/TestVersion.java
diff --git a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
similarity index 91%
rename from cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
rename to cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
index a5961ad..ca8a3cd 100644
--- a/cassandra-integration-tests/src/test/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
+++ b/cassandra-integration-tests/src/testFixtures/java/org/apache/cassandra/sidecar/common/testing/TestVersionSupplier.java
@@ -28,8 +28,8 @@ 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.
  *
- * <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
+ * <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
  *
  * <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.
diff --git a/cassandra40/build.gradle b/cassandra40/build.gradle
index ee71e39..15e28e5 100644
--- a/cassandra40/build.gradle
+++ b/cassandra40/build.gradle
@@ -14,4 +14,5 @@ repositories {
 
 dependencies {
     compile project(":common")
+    implementation('org.jetbrains:annotations:23.0.0')
 }
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 d742949..be98d32 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,10 +18,15 @@
 
 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.JmxClient;
 import org.apache.cassandra.sidecar.common.MinimumVersion;
+import org.apache.cassandra.sidecar.common.NodeStatus;
+import org.apache.cassandra.sidecar.common.StorageOperations;
 
 /**
  * Factory to produce the 4.0 adapter
@@ -29,9 +34,30 @@ import org.apache.cassandra.sidecar.common.MinimumVersion;
 @MinimumVersion("4.0.0")
 public class Cassandra40Factory implements ICassandraFactory
 {
+
+    /**
+     * Returns a new adapter for Cassandra 4.0 clusters.
+     *
+     * @param session the session to the Cassandra database
+     * @param jmxClient  the JMX client to connect to the Cassandra database
+     * @return a new adapter for the 4.0 clusters
+     */
     @Override
-    public ICassandraAdapter create(CQLSession session)
+    public ICassandraAdapter create(CQLSession session, JmxClient jmxClient)
     {
-        return () -> null;
+        return new ICassandraAdapter()
+        {
+            @Override
+            public List<NodeStatus> getStatus()
+            {
+                return null;
+            }
+
+            @Override
+            public StorageOperations storageOperations()
+            {
+                return new Cassandra40StorageOperations(jmxClient);
+            }
+        };
     }
 }
diff --git a/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40StorageOperations.java b/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40StorageOperations.java
new file mode 100644
index 0000000..a64b866
--- /dev/null
+++ b/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/Cassandra40StorageOperations.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cassandra40;
+
+import java.util.Map;
+
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.StorageOperations;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * An implementation of the {@link StorageOperations} that interfaces with Cassandra 4.0
+ */
+public class Cassandra40StorageOperations implements StorageOperations
+{
+    private static final String STORAGE_SERVICE_OBJ_NAME = "org.apache.cassandra.db:type=StorageService";
+    private final JmxClient jmxClient;
+
+    /**
+     * Creates a new instance with the provided {@link JmxClient}
+     *
+     * @param jmxClient the JMX client used to communicate with the Cassandra instance
+     */
+    public Cassandra40StorageOperations(JmxClient jmxClient)
+    {
+        this.jmxClient = jmxClient;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void takeSnapshot(@NotNull String tag, @NotNull String keyspace, @NotNull String table,
+                             @Nullable Map<String, String> options)
+    {
+        requireNonNull(tag, "snapshot tag must be non-null");
+        requireNonNull(keyspace, "keyspace for the  must be non-null");
+        requireNonNull(table, "table must be non-null");
+        jmxClient.proxy(StorageJmxOperations.class, STORAGE_SERVICE_OBJ_NAME)
+                 .takeSnapshot(tag, options, keyspace + "." + table);
+    }
+}
diff --git a/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/StorageJmxOperations.java b/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/StorageJmxOperations.java
new file mode 100644
index 0000000..3e9250a
--- /dev/null
+++ b/cassandra40/src/main/java/org/apache/cassandra/sidecar/cassandra40/StorageJmxOperations.java
@@ -0,0 +1,18 @@
+package org.apache.cassandra.sidecar.cassandra40;
+
+import java.util.Map;
+
+/**
+ * An interface that pulls a method from the Cassandra Storage Service Proxy
+ */
+public interface StorageJmxOperations
+{
+    /**
+     * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
+     *
+     * @param tag      the tag given to the snapshot; may not be null or empty
+     * @param options  map of options, for example ttl, skipFlush
+     * @param entities list of keyspaces / tables in the form of empty | ks1 ks2 ... | ks1.cf1,ks2.cf2,...
+     */
+    void takeSnapshot(String tag, Map<String, String> options, String... entities);
+}
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSession.java b/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSession.java
index b3e82b5..37b2abe 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSession.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/CQLSession.java
@@ -37,7 +37,7 @@ import com.datastax.driver.core.policies.RoundRobinPolicy;
 import com.datastax.driver.core.policies.WhiteListPolicy;
 
 /**
- * Represents a connection to Cassandra cluster. Currently supports returning the local connection only as
+ * Represents a connection to Cassandra cluster. Currently, supports returning the local connection only as
  * defined in the Configuration.
  */
 public class CQLSession
@@ -48,11 +48,11 @@ public class CQLSession
     private Session localSession;
     private final InetSocketAddress inet;
     private final WhiteListPolicy wlp;
-    private NettyOptions nettyOptions;
-    private QueryOptions queryOptions;
-    private ReconnectionPolicy reconnectionPolicy;
+    private final NettyOptions nettyOptions;
+    private final QueryOptions queryOptions;
+    private final ReconnectionPolicy reconnectionPolicy;
 
-    public CQLSession(String host, Integer port, Integer healthCheckFrequency)
+    public CQLSession(String host, int port, long healthCheckFrequency)
     {
         // this was originally using unresolved Inet addresses, but it would fail when trying to
         // connect to a docker container
@@ -75,7 +75,7 @@ public class CQLSession
     }
 
     /**
-     * Provides a Session connected only to the local node from configuration. If null it means the the connection was
+     * Provides a Session connected only to the local node from configuration. If null it means the connection was
      * not able to be established. The session still might throw a NoHostAvailableException if the local host goes
      * offline or otherwise unavailable.
      *
@@ -122,6 +122,14 @@ public class CQLSession
         return localSession;
     }
 
+    /**
+     * @return the address of the instance where the session is established
+     */
+    public InetSocketAddress inet()
+    {
+        return inet;
+    }
+
     public synchronized void close()
     {
         if (localSession != null)
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java b/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java
index ee3b0ce..0407a4d 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/CassandraAdapterDelegate.java
@@ -46,6 +46,7 @@ import org.jetbrains.annotations.NotNull;
 public class CassandraAdapterDelegate implements ICassandraAdapter, Host.StateListener
 {
     private final CQLSession cqlSession;
+    private final JmxClient jmxClient;
     private final CassandraVersionProvider versionProvider;
     private volatile Session session;
     private SimpleCassandraVersion currentVersion;
@@ -56,10 +57,11 @@ public class CassandraAdapterDelegate implements ICassandraAdapter, Host.StateLi
     private final AtomicBoolean registered = new AtomicBoolean(false);
     private final AtomicBoolean isHealthCheckActive = new AtomicBoolean(false);
 
-    public CassandraAdapterDelegate(CassandraVersionProvider provider, CQLSession cqlSession)
+    public CassandraAdapterDelegate(CassandraVersionProvider provider, CQLSession cqlSession, JmxClient jmxClient)
     {
-        this.cqlSession = cqlSession;
         this.versionProvider = provider;
+        this.cqlSession = cqlSession;
+        this.jmxClient = jmxClient;
     }
 
     private void maybeRegisterHostListener(@NotNull Session session)
@@ -150,7 +152,7 @@ public class CassandraAdapterDelegate implements ICassandraAdapter, Host.StateLi
             if (!newVersion.equals(currentVersion))
             {
                 currentVersion = newVersion;
-                adapter = versionProvider.getCassandra(version).create(cqlSession);
+                adapter = versionProvider.getCassandra(version).create(cqlSession, jmxClient);
                 logger.info("Cassandra version change detected. New adapter loaded: {}", adapter);
             }
             logger.debug("Cassandra version {}", version);
@@ -174,6 +176,12 @@ public class CassandraAdapterDelegate implements ICassandraAdapter, Host.StateLi
         return adapter.getStatus();
     }
 
+    @Override
+    public StorageOperations storageOperations()
+    {
+        return adapter.storageOperations();
+    }
+
     @Override
     public void onAdd(Host host)
     {
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java b/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java
index 9412df6..cf6c7bb 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java
@@ -30,4 +30,9 @@ public interface ICassandraAdapter
 {
     List<NodeStatus> getStatus();
 
+    /**
+     * @return the {@link StorageOperations} implementation for the Cassandra cluster
+     */
+    StorageOperations storageOperations();
+
 }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraFactory.java b/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraFactory.java
index d28a859..84c43e8 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraFactory.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraFactory.java
@@ -23,5 +23,12 @@ package org.apache.cassandra.sidecar.common;
  */
 public interface ICassandraFactory
 {
-    ICassandraAdapter create(CQLSession session);
+    /**
+     * Creates a new {@link ICassandraAdapter} with the provided {@link CQLSession} and {@link JmxClient}
+     *
+     * @param session the session to the Cassandra database
+     * @param client  the JMX client to connect to the Cassandra database
+     * @return a {@link ICassandraAdapter}
+     */
+    ICassandraAdapter create(CQLSession session, JmxClient client);
 }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java b/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
new file mode 100644
index 0000000..28cee13
--- /dev/null
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
@@ -0,0 +1,209 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.rmi.server.RMIClientSocketFactory;
+import java.rmi.server.RMISocketFactory;
+import java.util.HashMap;
+import java.util.Map;
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.Notification;
+import javax.management.NotificationListener;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnectionNotification;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+import javax.rmi.ssl.SslRMIClientSocketFactory;
+
+import org.jetbrains.annotations.VisibleForTesting;
+
+/**
+ * A simple wrapper around a JMX connection that makes it easier to get proxy instances.
+ */
+public class JmxClient implements NotificationListener
+{
+    public static final String JMX_SERVICE_URL_FMT = "service:jmx:rmi://%s/jndi/rmi://%s:%d/jmxrmi";
+    public static final String REGISTRY_CONTEXT_SOCKET_FACTORY = "com.sun.jndi.rmi.factory.socket";
+    private final JMXServiceURL jmxServiceURL;
+    private MBeanServerConnection mBeanServerConnection;
+    private final Map<String, Object> jmxEnv;
+    private boolean connected = false;
+
+    /**
+     * Creates a new client with the provided {@code host} and {@code port}.
+     *
+     * @param host the host of the JMX service
+     * @param port the port of the JMX service
+     */
+    public JmxClient(String host, int port)
+    {
+        this(host, port, null, null, false);
+    }
+
+    /**
+     * Creates a new client with the provided parameters
+     *
+     * @param host      the host of the JMX service
+     * @param port      the port of the JMX service
+     * @param role      the JMX role used for authentication
+     * @param password  the JMX role password used for authentication
+     * @param enableSSl true if SSL is enabled for JMX, false otherwise
+     */
+    public JmxClient(String host, int port, String role, String password, boolean enableSSl)
+    {
+        this(buildJmxServiceURL(host, port), role, password, enableSSl);
+    }
+
+    @VisibleForTesting
+    JmxClient(JMXServiceURL jmxServiceURL)
+    {
+        this(jmxServiceURL, null, null, false);
+    }
+
+    @VisibleForTesting
+    JmxClient(JMXServiceURL jmxServiceURL, String role, String password)
+    {
+        this(jmxServiceURL, role, password, false);
+    }
+
+    private JmxClient(JMXServiceURL jmxServiceURL, String role, String password, boolean enableSsl)
+    {
+        this.jmxServiceURL = jmxServiceURL;
+
+        jmxEnv = new HashMap<>();
+        if (role != null && password != null)
+        {
+            String[] credentials = new String[]{ role, password };
+            jmxEnv.put(JMXConnector.CREDENTIALS, credentials);
+        }
+        jmxEnv.put(REGISTRY_CONTEXT_SOCKET_FACTORY, getRMIClientSocketFactory(enableSsl));
+    }
+
+    /**
+     * Returns a proxy for a Standard MBean in a local or remote MBean Server.
+     *
+     * @param clientClass the management interface that the MBean exports, which will
+     *                    also be implemented by the returned proxy
+     * @param remoteName  the name of the MBean within {@code connection} to forward to
+     * @param <C>         the type of the proxy client
+     * @return the proxy for a Standard MBean in a local or remote MBean Server
+     */
+    public <C> C proxy(Class<C> clientClass, String remoteName)
+    {
+        checkConnection();
+        try
+        {
+            // NOTE: We get a new proxy each time we need one (much of the underlying construction is
+            // cached by the JMX infrastructure, so we believe this to not be terribly resource-intensive
+            ObjectName name = new ObjectName(remoteName);
+            return JMX.newMBeanProxy(mBeanServerConnection, name, clientClass);
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(String.format("Invalid remote object name '%s'", remoteName), e);
+        }
+    }
+
+    private RMIClientSocketFactory getRMIClientSocketFactory(boolean enableSsl)
+    {
+        return enableSsl
+               ? new SslRMIClientSocketFactory()
+               : RMISocketFactory.getDefaultSocketFactory();
+    }
+
+    private synchronized void checkConnection()
+    {
+        if (!this.connected)
+        {
+            connect();
+        }
+    }
+
+    private void connect()
+    {
+        try
+        {
+            JMXConnector jmxConnector = JMXConnectorFactory.connect(jmxServiceURL, jmxEnv);
+            jmxConnector.addConnectionNotificationListener(this, null, null);
+            mBeanServerConnection = jmxConnector.getMBeanServerConnection();
+            connected = true;
+        }
+        catch (IOException iox)
+        {
+            connected = false;
+            throw new RuntimeException(String.format("Failed to connect to JMX endpoint %s", jmxServiceURL),
+                                       iox);
+        }
+    }
+
+    public void handleNotification(Notification notification, Object handback)
+    {
+        if (notification instanceof JMXConnectionNotification)
+        {
+            JMXConnectionNotification connectNotice = (JMXConnectionNotification) notification;
+            final String type = connectNotice.getType();
+            if (type.equals(JMXConnectionNotification.CLOSED) ||
+                type.equals(JMXConnectionNotification.FAILED) ||
+                type.equals(JMXConnectionNotification.NOTIFS_LOST) ||
+                type.equals(JMXConnectionNotification.OPENED))
+            {
+                boolean justConnected = type.equals(JMXConnectionNotification.OPENED);
+                synchronized (this)
+                {
+                    this.connected = justConnected;
+                }
+            }
+        }
+    }
+
+    /**
+     * @return true if JMX is connected, false otherwise
+     */
+    public boolean isConnected()
+    {
+        return connected;
+    }
+
+    private static JMXServiceURL buildJmxServiceURL(String host, int port)
+    {
+        if (host == null)
+            return null;
+
+        if (host.contains(":"))
+        {
+            host = "[" + host + "]";
+            // Use square brackets to surround IPv6 addresses to fix CASSANDRA-7669 and CASSANDRA-17581
+        }
+        try
+        {
+            return new JMXServiceURL(String.format(JMX_SERVICE_URL_FMT, host, host, port));
+        }
+        catch (MalformedURLException e)
+        {
+            String errorMessage = String.format("Unable to build JMXServiceURL for host=%s, port=%d",
+                                                host, port);
+            throw new RuntimeException(errorMessage, e);
+        }
+    }
+}
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/MockCassandraFactory.java b/common/src/main/java/org/apache/cassandra/sidecar/common/MockCassandraFactory.java
index a3aca93..5e6cd68 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/MockCassandraFactory.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/MockCassandraFactory.java
@@ -24,7 +24,7 @@ package org.apache.cassandra.sidecar.common;
 public class MockCassandraFactory implements ICassandraFactory
 {
     @Override
-    public ICassandraAdapter create(CQLSession session)
+    public ICassandraAdapter create(CQLSession session, JmxClient jmxClient)
     {
         return null;
     }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java b/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
similarity index 51%
copy from common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java
copy to common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
index 9412df6..8bc505b 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/ICassandraAdapter.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/StorageOperations.java
@@ -18,16 +18,24 @@
 
 package org.apache.cassandra.sidecar.common;
 
-import java.util.List;
+import java.util.Map;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Core Cassandra Adapter interface
- * For now, this is just a placeholder.  We will most likely want to define the interface to returns bits such as
- * getCompaction(), getClusterMembership, etc, which return interfaces such as ICompaction, IClusterMembership.
- * We will need different implementations due to the slow move away from JMX towards CQL for some, but not all, actions.
+ * An interface that defines interactions with the storage system in Cassandra.
  */
-public interface ICassandraAdapter
+public interface StorageOperations
 {
-    List<NodeStatus> getStatus();
-
+    /**
+     * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
+     *
+     * @param tag      the tag given to the snapshot; may not be null or empty
+     * @param keyspace the keyspace in the Cassandra database to use for the snapshot
+     * @param table    the table in the Cassandra database to use for the snapshot
+     * @param options  map of options, for example ttl, skipFlush
+     */
+    void takeSnapshot(@NotNull String tag, @NotNull String keyspace, @NotNull String table,
+                      @Nullable Map<String, String> options);
 }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V30.java b/common/src/main/java/org/apache/cassandra/sidecar/mocks/V30.java
index b14a6f3..7dcf7ef 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V30.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/mocks/V30.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.sidecar.mocks;
 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.JmxClient;
 import org.apache.cassandra.sidecar.common.MinimumVersion;
 
 /**
@@ -30,7 +31,7 @@ import org.apache.cassandra.sidecar.common.MinimumVersion;
 public class V30 implements ICassandraFactory
 {
     @Override
-    public ICassandraAdapter create(CQLSession session)
+    public ICassandraAdapter create(CQLSession session, JmxClient jmxClient)
     {
         return null;
     }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V40.java b/common/src/main/java/org/apache/cassandra/sidecar/mocks/V40.java
index 2d5e6e2..9b2398e 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V40.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/mocks/V40.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.sidecar.mocks;
 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.JmxClient;
 import org.apache.cassandra.sidecar.common.MinimumVersion;
 
 /**
@@ -30,7 +31,7 @@ import org.apache.cassandra.sidecar.common.MinimumVersion;
 public class V40 implements ICassandraFactory
 {
     @Override
-    public ICassandraAdapter create(CQLSession session)
+    public ICassandraAdapter create(CQLSession session, JmxClient jmxClient)
     {
         return null;
     }
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V41.java b/common/src/main/java/org/apache/cassandra/sidecar/mocks/V41.java
index 35e5973..130a6ea 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/mocks/V41.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/mocks/V41.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.sidecar.mocks;
 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.JmxClient;
 import org.apache.cassandra.sidecar.common.MinimumVersion;
 
 /**
@@ -30,7 +31,7 @@ import org.apache.cassandra.sidecar.common.MinimumVersion;
 public class V41 implements ICassandraFactory
 {
     @Override
-    public ICassandraAdapter create(CQLSession session)
+    public ICassandraAdapter create(CQLSession session, JmxClient jmxClient)
     {
         return null;
     }
diff --git a/common/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java b/common/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java
new file mode 100644
index 0000000..9a88c5b
--- /dev/null
+++ b/common/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java
@@ -0,0 +1,251 @@
+/*
+ * 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;
+
+import java.lang.management.ManagementFactory;
+import java.net.MalformedURLException;
+import java.rmi.registry.LocateRegistry;
+import java.rmi.registry.Registry;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.logging.Logger;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnectorServer;
+import javax.management.remote.JMXConnectorServerFactory;
+import javax.management.remote.JMXServiceURL;
+
+import com.google.common.collect.Sets;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.platform.commons.util.Preconditions;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+
+
+/***
+ * In order to support multiple versions of Cassandra in the Sidecar, we would like to avoid depending directly on
+ * any Cassandra code.
+ * Additionally, whe would like to avoid copy/pasting the entire MBean interface classes into the Sidecar.
+ * This test exists to prove out some assumptions about using matching sub-interfaces (or even functional interfaces)
+ * to make JMX calls. This particular call happens to match the signature of the `importNewSSTables` method on
+ * StorageServiceProxy in C* 4.0.
+ */
+public class JmxClientTest
+{
+
+    private static final JMXServiceURL serviceURL;
+    private static final String objectName = "org.apache.cassandra.jmx:type=ExtendedImport";
+    public static final int PROXIES_TO_TEST = 10_000;
+    private static StorageService importMBean;
+    private static JMXConnectorServer jmxServer;
+    private static MBeanServer mbs;
+    private static Registry registry;
+
+    @BeforeAll
+    public static void setUp() throws Exception
+    {
+        System.setProperty("java.rmi.server.randomIds", "true");
+        String passwordFile = Objects.requireNonNull(JmxClientTest.class
+                                                     .getClassLoader()
+                                                     .getResource("testJmxPassword.properties")).getPath();
+        Map<String, String> env = new HashMap<>();
+        env.put("jmx.remote.x.password.file", passwordFile);
+        registry = LocateRegistry.createRegistry(9999);
+        mbs = ManagementFactory.getPlatformMBeanServer();
+        jmxServer = JMXConnectorServerFactory.newJMXConnectorServer(serviceURL, env, mbs);
+        jmxServer.start();
+        importMBean = new StorageService();
+        mbs.registerMBean(importMBean, new ObjectName(objectName));
+    }
+
+    @AfterAll
+    public static void tearDown() throws Exception
+    {
+        jmxServer.stop();
+        final ObjectName name = new ObjectName(objectName);
+        if (mbs.isRegistered(name))
+        {
+            mbs.unregisterMBean(name);
+        }
+        UnicastRemoteObject.unexportObject(registry, true);
+        registry = null;
+    }
+
+    @BeforeEach
+    public void setup()
+    {
+        importMBean.shouldSucceed = true;
+    }
+
+    @Test
+    public void testCanCallMethodWithoutEntireInterface()
+    {
+        JmxClient client = new JmxClient(serviceURL, "controlRole", "password");
+        List<String> result = client.proxy(Import.class, objectName)
+                                    .importNewSSTables(Sets.newHashSet("foo", "bar"), true,
+                                                       true, true, true, true,
+                                                       true);
+        assertThat(result.size()).isEqualTo(0);
+    }
+
+    @Test
+    public void testCanCallMethodWithoutEntireInterfaceGetResults()
+    {
+        importMBean.shouldSucceed = false;
+        JmxClient client = new JmxClient(serviceURL, "controlRole", "password");
+        final HashSet<String> srcPaths = Sets.newHashSet("foo", "bar");
+        final List<String> failedDirs = client.proxy(Import.class, objectName)
+                                              .importNewSSTables(srcPaths, true,
+                                                                 true, true, true, true,
+                                                                 true);
+        assertThat(failedDirs.size()).isEqualTo(2);
+        assertThat(failedDirs.toArray()).isEqualTo(srcPaths.toArray());
+    }
+
+    @Test
+    public void testCallWithoutCredentialsFails()
+    {
+        assertThatExceptionOfType(SecurityException.class)
+        .isThrownBy(() ->
+                    {
+                        JmxClient client = new JmxClient(serviceURL);
+                        client.proxy(Import.class, objectName)
+                              .importNewSSTables(Sets.newHashSet("foo", "bar"),
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true);
+                    });
+    }
+
+    @Test
+    public void testDisconnectReconnect() throws Exception
+    {
+        JmxClient client = new JmxClient(serviceURL, "controlRole", "password");
+        assertThat(client.isConnected()).isFalse();
+        List<String> result = client.proxy(Import.class, objectName)
+                                    .importNewSSTables(
+                                    Sets.newHashSet("foo", "bar"), true, true, true,
+                                    true, true,
+                                    true);
+        assertThat(client.isConnected()).isTrue();
+        assertThat(result.size()).isEqualTo(0);
+
+        tearDown();
+        setUp();
+
+        result = client.proxy(Import.class, objectName)
+                       .importNewSSTables(
+                       Sets.newHashSet("foo", "bar"), true, true, true,
+                       true, true,
+                       true);
+        assertThat(result.size()).isEqualTo(0);
+    }
+
+    @Test
+    public void testLotsOfProxies()
+    {
+        JmxClient client = new JmxClient(serviceURL, "controlRole", "password");
+        for (int i = 0; i < PROXIES_TO_TEST; i++)
+        {
+            List<String> result = client.proxy(Import.class, objectName)
+                                        .importNewSSTables(
+                                        Sets.newHashSet("foo", "bar"), true, true, true,
+                                        true, true,
+                                        true);
+            assertThat(result).isNotNull();
+        }
+    }
+
+    /**
+     * Simulates to C*'s `nodetool import` call
+     */
+    public interface Import
+    {
+        List<String> importNewSSTables(Set<String> srcPaths, boolean resetLevel, boolean clearRepaired,
+                                       boolean verifySSTables, boolean verifyTokens, boolean invalidateCaches,
+                                       boolean extendedVerify);
+    }
+
+    /**
+     * Simulates the larger Storage Service MBean interface
+     */
+    public interface StorageServiceMBean
+    {
+        List<String> importNewSSTables(Set<String> srcPaths, boolean resetLevel, boolean clearRepaired,
+                                       boolean verifySSTables, boolean verifyTokens, boolean invalidateCaches,
+                                       boolean extendedVerify);
+
+        void someOtherMethod(String helloString);
+    }
+
+    /**
+     * An implementation of our mock StorageServiceMBean
+     */
+    public static class StorageService implements StorageServiceMBean
+    {
+
+        private static final Logger logger = Logger.getLogger(StorageService.class.getSimpleName());
+        public boolean shouldSucceed = true;
+
+        @Override
+        public List<String> importNewSSTables(Set<String> srcPaths, boolean resetLevel, boolean clearRepaired,
+                                              boolean verifySSTables, boolean verifyTokens,
+                                              boolean invalidateCaches, boolean extendedVerify)
+        {
+            Preconditions.notNull(srcPaths, "Source Paths missing");
+            if (shouldSucceed)
+            {
+                return Collections.emptyList();
+            }
+            return Arrays.asList(srcPaths.toArray(new String[0]));
+        }
+
+        @Override
+        public void someOtherMethod(String helloString)
+        {
+            logger.info(helloString);
+        }
+    }
+
+    static
+    {
+        try
+        {
+            serviceURL = new JMXServiceURL("service:jmx:rmi://localhost/jndi/rmi://localhost:9999/jmxrmi");
+        }
+        catch (MalformedURLException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/common/src/test/resources/testJmxPassword.properties b/common/src/test/resources/testJmxPassword.properties
new file mode 100644
index 0000000..cfa135f
--- /dev/null
+++ b/common/src/test/resources/testJmxPassword.properties
@@ -0,0 +1 @@
+controlRole password
diff --git a/spotbugs-exclude.xml b/spotbugs-exclude.xml
index ee76f3c..e9467b6 100644
--- a/spotbugs-exclude.xml
+++ b/spotbugs-exclude.xml
@@ -23,4 +23,13 @@
         <Bug pattern="DMI_HARDCODED_ABSOLUTE_FILENAME" />
     </Match>
 
-</FindBugsFilter>
\ No newline at end of file
+    <!--
+    Ignore NM_SAME_SIMPLE_NAME_AS_SUPERCLASS for our own provided CassandraContainer class. This
+    class is internal to the test and won't be exposed to the testing classes directly.
+     -->
+    <Match>
+        <Class name="org.apache.cassandra.sidecar.common.containers.CassandraContainer" />
+        <Bug pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS" />
+    </Match>
+
+</FindBugsFilter>
diff --git a/src/main/dist/conf/sidecar.yaml b/src/main/dist/conf/sidecar.yaml
index a718f75..c5254d4 100644
--- a/src/main/dist/conf/sidecar.yaml
+++ b/src/main/dist/conf/sidecar.yaml
@@ -6,10 +6,20 @@ cassandra_instances:
     host: localhost1
     port: 9042
     data_dirs: /cassandra/d1/data, /cassandra/d2/data
+    jmx_host: 127.0.0.1
+    jmx_port: 7100
+    jmx_ssl_enabled: false
+#    jmx_role:
+#    jmx_role_password:
   - id: 2
     host: localhost2
     port: 9042
     data_dirs: /cassandra/d3/data, /cassandra/d4/data
+    jmx_host: 127.0.0.1
+    jmx_port: 7200
+    jmx_ssl_enabled: false
+#    jmx_role:
+#    jmx_role_password:
 
 sidecar:
   - host: 0.0.0.0
diff --git a/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java b/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
index 25924fb..d788f7b 100644
--- a/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
+++ b/src/main/java/org/apache/cassandra/sidecar/CassandraSidecarDaemon.java
@@ -56,8 +56,14 @@ public class CassandraSidecarDaemon
         banner(System.out);
         validate();
         logger.info("Starting Cassandra Sidecar on {}:{}", config.getHost(), config.getPort());
-        server.listen(config.getPort(), config.getHost());
-        healthCheckTimerId = vertx.setPeriodic(config.getHealthCheckFrequencyMillis(), this::healthCheck);
+        server.listen(config.getPort(), config.getHost())
+              .onSuccess(p -> {
+                  // Run a health check after start up
+                  healthCheck();
+                  // Configure the periodic timer to run subsequent health checks configured
+                  // by the config.getHealthCheckFrequencyMillis() interval
+                  updateHealthChecker(config.getHealthCheckFrequencyMillis());
+              });
     }
 
     public void stop()
@@ -65,6 +71,10 @@ public class CassandraSidecarDaemon
         logger.info("Stopping Cassandra Sidecar");
         server.close();
         vertx.cancelTimer(healthCheckTimerId);
+        config.getInstancesConfig()
+              .instances()
+              .forEach(instanceMetadata ->
+                       vertx.executeBlocking(promise -> instanceMetadata.session().close()));
     }
 
     private void banner(PrintStream out)
@@ -101,18 +111,34 @@ public class CassandraSidecarDaemon
 
     }
 
+    /**
+     * Updates the health check frequency to the provided {@code healthCheckFrequencyMillis} value
+     *
+     * @param healthCheckFrequencyMillis the new health check frequency in milliseconds
+     */
+    public void updateHealthChecker(long healthCheckFrequencyMillis)
+    {
+        if (healthCheckTimerId > 0)
+        {
+            // Stop existing timer
+            vertx.cancelTimer(healthCheckTimerId);
+            logger.info("Stopped health check timer with timerId={}", healthCheckTimerId);
+        }
+        // TODO: when upgrading to latest vertx version, we can set an initial delay, and the periodic delay
+        healthCheckTimerId = vertx.setPeriodic(healthCheckFrequencyMillis, t -> healthCheck());
+        logger.info("Started health check with frequency={} and timerId={}",
+                    healthCheckFrequencyMillis, healthCheckTimerId);
+    }
+
     /**
      * Checks the health of every instance configured in the {@link Configuration#getInstancesConfig()}.
      * The health check is executed in a blocking thread to prevent the event-loop threads from blocking.
-     *
-     * @param timerId the ID of the periodic timer
      */
-    private void healthCheck(Long timerId)
+    private void healthCheck()
     {
-        config.getInstancesConfig()
-              .instances()
-              .forEach(instanceMetadata ->
-                       vertx.executeBlocking(promise -> instanceMetadata.delegate().healthCheck()));
+        config.getInstancesConfig().instances()
+              .forEach((instanceMetadata) ->
+                       vertx.executeBlocking((promise) -> instanceMetadata.delegate().healthCheck()));
     }
 
 
diff --git a/src/main/java/org/apache/cassandra/sidecar/Configuration.java b/src/main/java/org/apache/cassandra/sidecar/Configuration.java
index 6ed92a8..538b25d 100644
--- a/src/main/java/org/apache/cassandra/sidecar/Configuration.java
+++ b/src/main/java/org/apache/cassandra/sidecar/Configuration.java
@@ -35,10 +35,10 @@ public class Configuration
     private final Integer port;
 
     /* Sidecar's listen address */
-    private String host;
+    private final String host;
 
-    /* Healthcheck frequency in miilis */
-    private final Integer healthCheckFrequencyMillis;
+    /* Healthcheck frequency in millis */
+    private final long healthCheckFrequencyMillis;
 
     /* SSL related settings */
     @Nullable
@@ -63,7 +63,7 @@ public class Configuration
 
     private final ValidationConfiguration validationConfiguration;
 
-    public Configuration(InstancesConfig instancesConfig, String host, Integer port, Integer healthCheckFrequencyMillis,
+    public Configuration(InstancesConfig instancesConfig, String host, Integer port, long healthCheckFrequencyMillis,
                          boolean isSslEnabled, @Nullable String keyStorePath, @Nullable String keyStorePassword,
                          @Nullable String trustStorePath, @Nullable String trustStorePassword,
                          long rateLimitStreamRequestsPerSecond, long throttleTimeoutInSeconds,
@@ -86,15 +86,22 @@ public class Configuration
     }
 
     /**
-     * Get Cassandra Instances config
-     *
-     * @return
+     * @return the Cassandra Instances config
      */
     public InstancesConfig getInstancesConfig()
     {
         return instancesConfig;
     }
 
+    /**
+     *
+     * @return the Cassandra validation configuration
+     */
+    public ValidationConfiguration getValidationConfiguration()
+    {
+        return validationConfiguration;
+    }
+
     /**
      *  Sidecar's listen address
      *
@@ -120,7 +127,7 @@ public class Configuration
      *
      * @return
      */
-    public Integer getHealthCheckFrequencyMillis()
+    public long getHealthCheckFrequencyMillis()
     {
         return healthCheckFrequencyMillis;
     }
diff --git a/src/main/java/org/apache/cassandra/sidecar/MainModule.java b/src/main/java/org/apache/cassandra/sidecar/MainModule.java
index 3ef46b3..86132f3 100644
--- a/src/main/java/org/apache/cassandra/sidecar/MainModule.java
+++ b/src/main/java/org/apache/cassandra/sidecar/MainModule.java
@@ -19,24 +19,8 @@
 package org.apache.cassandra.sidecar;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-import java.util.function.UnaryOperator;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.SidecarRateLimiter;
-import org.apache.commons.configuration2.HierarchicalConfiguration;
-import org.apache.commons.configuration2.YAMLConfiguration;
-import org.apache.commons.configuration2.ex.ConfigurationException;
-import org.apache.commons.configuration2.tree.ImmutableNode;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.inject.AbstractModule;
 import com.google.inject.Provides;
@@ -53,14 +37,9 @@ import io.vertx.ext.web.handler.LoggerHandler;
 import io.vertx.ext.web.handler.StaticHandler;
 import org.apache.cassandra.sidecar.cassandra40.Cassandra40Factory;
 import org.apache.cassandra.sidecar.cluster.InstancesConfig;
-import org.apache.cassandra.sidecar.cluster.InstancesConfigImpl;
-import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
-import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadataImpl;
-import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
 import org.apache.cassandra.sidecar.common.data.QualifiedTableName;
 import org.apache.cassandra.sidecar.common.utils.ValidationConfiguration;
-import org.apache.cassandra.sidecar.common.utils.YAMLValidationConfiguration;
 import org.apache.cassandra.sidecar.routes.CassandraHealthService;
 import org.apache.cassandra.sidecar.routes.FileStreamHandler;
 import org.apache.cassandra.sidecar.routes.HealthService;
@@ -72,35 +51,11 @@ import org.jboss.resteasy.plugins.server.vertx.VertxRegistry;
 import org.jboss.resteasy.plugins.server.vertx.VertxRequestHandler;
 import org.jboss.resteasy.plugins.server.vertx.VertxResteasyDeployment;
 
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_ALLOWED_CHARS_FOR_COMPONENT_NAME;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_ALLOWED_CHARS_FOR_DIRECTORY;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_ALLOWED_CHARS_FOR_RESTRICTED_COMPONENT_NAME;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_FORBIDDEN_KEYSPACES;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INPUT_VALIDATION;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCES;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_DATA_DIRS;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_HOST;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_ID;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_PORT;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.HEALTH_CHECK_FREQ;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.HOST;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.KEYSTORE_PASSWORD;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.KEYSTORE_PATH;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.PORT;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.SSL_ENABLED;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.STREAM_REQUESTS_PER_SEC;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.THROTTLE_DELAY_SEC;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.THROTTLE_TIMEOUT_SEC;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.TRUSTSTORE_PASSWORD;
-import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.TRUSTSTORE_PATH;
-
 /**
  * Provides main binding for more complex Guice dependencies
  */
 public class MainModule extends AbstractModule
 {
-    private static final Logger logger = LoggerFactory.getLogger(MainModule.class);
     private static final String API_V1_VERSION = "/api/v1";
 
     @Override
@@ -209,110 +164,25 @@ public class MainModule extends AbstractModule
 
     @Provides
     @Singleton
-    public Configuration configuration(YAMLConfiguration yamlConf, InstancesConfig instancesConfig,
-                                       ValidationConfiguration validationConfiguration)
+    public Configuration configuration(CassandraVersionProvider cassandraVersionProvider)
     throws IOException
     {
-        return new Configuration.Builder()
-               .setInstancesConfig(instancesConfig)
-               .setHost(yamlConf.get(String.class, HOST))
-               .setPort(yamlConf.get(Integer.class, PORT))
-               .setHealthCheckFrequency(yamlConf.get(Integer.class, HEALTH_CHECK_FREQ))
-               .setKeyStorePath(yamlConf.get(String.class, KEYSTORE_PATH, null))
-               .setKeyStorePassword(yamlConf.get(String.class, KEYSTORE_PASSWORD, null))
-               .setTrustStorePath(yamlConf.get(String.class, TRUSTSTORE_PATH, null))
-               .setTrustStorePassword(yamlConf.get(String.class, TRUSTSTORE_PASSWORD, null))
-               .setSslEnabled(yamlConf.get(Boolean.class, SSL_ENABLED, false))
-               .setRateLimitStreamRequestsPerSecond(yamlConf.getLong(STREAM_REQUESTS_PER_SEC))
-               .setThrottleTimeoutInSeconds(yamlConf.getLong(THROTTLE_TIMEOUT_SEC))
-               .setThrottleDelayInSeconds(yamlConf.getLong(THROTTLE_DELAY_SEC))
-               .setValidationConfiguration(validationConfiguration)
-               .build();
-    }
-
-    @Provides
-    @Singleton
-    public InstancesConfig getInstancesConfig(YAMLConfiguration yamlConf, CassandraVersionProvider versionProvider)
-    {
-        return readInstancesConfig(yamlConf, versionProvider);
+        final String confPath = System.getProperty("sidecar.config", "file://./conf/config.yaml");
+        return YAMLSidecarConfiguration.of(confPath, cassandraVersionProvider);
     }
 
     @Provides
     @Singleton
-    public ValidationConfiguration validationConfiguration(YAMLConfiguration yamlConf)
-    {
-        org.apache.commons.configuration2.Configuration validation = yamlConf.subset(CASSANDRA_INPUT_VALIDATION);
-        Set<String> forbiddenKeyspaces = new HashSet<>(validation.getList(String.class,
-                                                                          CASSANDRA_FORBIDDEN_KEYSPACES,
-                                                                          Collections.emptyList()));
-        UnaryOperator<String> readString = key -> validation.get(String.class, key);
-        String allowedPatternForDirectory = readString.apply(CASSANDRA_ALLOWED_CHARS_FOR_DIRECTORY);
-        String allowedPatternForComponentName = readString.apply(CASSANDRA_ALLOWED_CHARS_FOR_COMPONENT_NAME);
-        String allowedPatternForRestrictedComponentName = readString
-                                                          .apply(CASSANDRA_ALLOWED_CHARS_FOR_RESTRICTED_COMPONENT_NAME);
-
-        return new YAMLValidationConfiguration(forbiddenKeyspaces,
-                                               allowedPatternForDirectory,
-                                               allowedPatternForComponentName,
-                                               allowedPatternForRestrictedComponentName);
-    }
-
-    @VisibleForTesting
-    public InstancesConfigImpl readInstancesConfig(YAMLConfiguration yamlConf, CassandraVersionProvider versionProvider)
+    public InstancesConfig getInstancesConfig(Configuration configuration)
     {
-        final int healthCheckFrequencyMillis = yamlConf.get(Integer.class, HEALTH_CHECK_FREQ);
-
-        /* Since we are supporting handling multiple instances in Sidecar optionally, we prefer reading single instance
-         * data over reading multiple instances section
-         */
-        org.apache.commons.configuration2.Configuration singleInstanceConf = yamlConf.subset(CASSANDRA_INSTANCE);
-        if (singleInstanceConf != null && !singleInstanceConf.isEmpty())
-        {
-            String host = singleInstanceConf.get(String.class, CASSANDRA_INSTANCE_HOST);
-            int port = singleInstanceConf.get(Integer.class, CASSANDRA_INSTANCE_PORT);
-            String dataDirs = singleInstanceConf.get(String.class, CASSANDRA_INSTANCE_DATA_DIRS);
-            CQLSession session = new CQLSession(host, port, healthCheckFrequencyMillis);
-            return new InstancesConfigImpl(Collections.singletonList(new InstanceMetadataImpl(1, host, port,
-                Collections.unmodifiableList(Arrays.asList(dataDirs.split(","))), session,
-                versionProvider, healthCheckFrequencyMillis)));
-        }
-
-        List<HierarchicalConfiguration<ImmutableNode>> instances = yamlConf.configurationsAt(CASSANDRA_INSTANCES);
-        final List<InstanceMetadata> instanceMetas = new LinkedList<>();
-        for (HierarchicalConfiguration<ImmutableNode> instance : instances)
-        {
-            int id = instance.get(Integer.class, CASSANDRA_INSTANCE_ID);
-            String host = instance.get(String.class, CASSANDRA_INSTANCE_HOST);
-            int port = instance.get(Integer.class, CASSANDRA_INSTANCE_PORT);
-            String dataDirs = instance.get(String.class, CASSANDRA_INSTANCE_DATA_DIRS);
-
-            CQLSession session = new CQLSession(host, port, healthCheckFrequencyMillis);
-            instanceMetas.add(new InstanceMetadataImpl(id, host, port,
-                Collections.unmodifiableList(Arrays.asList(dataDirs.split(","))), session, versionProvider,
-                healthCheckFrequencyMillis));
-        }
-        return new InstancesConfigImpl(instanceMetas);
+        return configuration.getInstancesConfig();
     }
 
     @Provides
     @Singleton
-    public YAMLConfiguration yamlConfiguration() throws ConfigurationException
+    public ValidationConfiguration validationConfiguration(Configuration configuration)
     {
-        final String confPath = System.getProperty("sidecar.config", "file://./conf/config.yaml");
-        logger.info("Reading configuration from {}", confPath);
-
-        try
-        {
-            URL url = new URL(confPath);
-            YAMLConfiguration yamlConf = new YAMLConfiguration();
-            InputStream stream = url.openStream();
-            yamlConf.read(stream);
-            return yamlConf;
-        }
-        catch (IOException e)
-        {
-            throw new ConfigurationException(String.format("Unable to parse cluster information from '%s'", confPath));
-        }
+        return configuration.getValidationConfiguration();
     }
 
     @Provides
diff --git a/src/main/java/org/apache/cassandra/sidecar/YAMLSidecarConfiguration.java b/src/main/java/org/apache/cassandra/sidecar/YAMLSidecarConfiguration.java
new file mode 100644
index 0000000..22b2452
--- /dev/null
+++ b/src/main/java/org/apache/cassandra/sidecar/YAMLSidecarConfiguration.java
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.UnaryOperator;
+
+import org.apache.commons.configuration2.HierarchicalConfiguration;
+import org.apache.commons.configuration2.YAMLConfiguration;
+import org.apache.commons.configuration2.ex.ConfigurationException;
+import org.apache.commons.configuration2.tree.ImmutableNode;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.cluster.InstancesConfig;
+import org.apache.cassandra.sidecar.cluster.InstancesConfigImpl;
+import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadataImpl;
+import org.apache.cassandra.sidecar.common.CQLSession;
+import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
+import org.apache.cassandra.sidecar.common.JmxClient;
+import org.apache.cassandra.sidecar.common.utils.ValidationConfiguration;
+import org.apache.cassandra.sidecar.common.utils.YAMLValidationConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_ALLOWED_CHARS_FOR_COMPONENT_NAME;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_ALLOWED_CHARS_FOR_DIRECTORY;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_ALLOWED_CHARS_FOR_RESTRICTED_COMPONENT_NAME;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_FORBIDDEN_KEYSPACES;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INPUT_VALIDATION;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCES;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_DATA_DIRS;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_HOST;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_ID;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_INSTANCE_PORT;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_JMX_HOST;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_JMX_PORT;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_JMX_ROLE;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_JMX_ROLE_PASSWORD;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.CASSANDRA_JMX_SSL_ENABLED;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.HEALTH_CHECK_FREQ;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.HOST;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.KEYSTORE_PASSWORD;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.KEYSTORE_PATH;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.PORT;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.SSL_ENABLED;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.STREAM_REQUESTS_PER_SEC;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.THROTTLE_DELAY_SEC;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.THROTTLE_TIMEOUT_SEC;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.TRUSTSTORE_PASSWORD;
+import static org.apache.cassandra.sidecar.utils.YAMLKeyConstants.TRUSTSTORE_PATH;
+
+/**
+ * A {@link Configuration} that is built from a YAML configuration file for Sidecar
+ */
+public class YAMLSidecarConfiguration extends Configuration
+{
+    private static final Logger logger = LoggerFactory.getLogger(YAMLSidecarConfiguration.class);
+
+    private YAMLSidecarConfiguration(InstancesConfig instancesConfig,
+                                     String host,
+                                     Integer port,
+                                     long healthCheckFrequencyMillis,
+                                     boolean isSslEnabled,
+                                     @Nullable String keyStorePath,
+                                     @Nullable String keyStorePassword,
+                                     @Nullable String trustStorePath,
+                                     @Nullable String trustStorePassword,
+                                     long rateLimitStreamRequestsPerSecond,
+                                     long throttleTimeoutInSeconds,
+                                     long throttleDelayInSeconds,
+                                     ValidationConfiguration validationConfiguration)
+    {
+        super(instancesConfig,
+              host,
+              port,
+              healthCheckFrequencyMillis,
+              isSslEnabled,
+              keyStorePath,
+              keyStorePassword,
+              trustStorePath,
+              trustStorePassword,
+              rateLimitStreamRequestsPerSecond,
+              throttleTimeoutInSeconds,
+              throttleDelayInSeconds,
+              validationConfiguration);
+    }
+
+    /**
+     * Returns a new {@link Configuration} built from the provided {@code confPath} YAML file and a
+     * {@code versionProvider}
+     *
+     * @param confPath        the path to the Sidecar YAML configuration file
+     * @param versionProvider a Cassandra version provider
+     * @return the {@link YAMLConfiguration} parsed from the YAML file
+     * @throws IOException when reading the configuration from file fails
+     */
+    public static Configuration of(String confPath, CassandraVersionProvider versionProvider) throws IOException
+    {
+        YAMLConfiguration yamlConf = yamlConfiguration(confPath);
+        long healthCheckFrequencyMillis = yamlConf.getLong(HEALTH_CHECK_FREQ, 1000);
+        ValidationConfiguration validationConfiguration = validationConfiguration(yamlConf);
+        InstancesConfig instancesConfig = instancesConfig(yamlConf, versionProvider, healthCheckFrequencyMillis);
+
+        return new YAMLSidecarConfiguration(instancesConfig,
+                                            yamlConf.get(String.class, HOST),
+                                            yamlConf.get(Integer.class, PORT),
+                                            healthCheckFrequencyMillis,
+                                            yamlConf.get(Boolean.class, SSL_ENABLED, false),
+                                            yamlConf.get(String.class, KEYSTORE_PATH, null),
+                                            yamlConf.get(String.class, KEYSTORE_PASSWORD, null),
+                                            yamlConf.get(String.class, TRUSTSTORE_PATH, null),
+                                            yamlConf.get(String.class, TRUSTSTORE_PASSWORD, null),
+                                            yamlConf.getLong(STREAM_REQUESTS_PER_SEC, 5000L),
+                                            yamlConf.getLong(THROTTLE_TIMEOUT_SEC, 10),
+                                            yamlConf.getLong(THROTTLE_DELAY_SEC, 5),
+                                            validationConfiguration);
+    }
+
+    /**
+     * Returns an object to read the YAML file from {@code confPath}.
+     *
+     * @param confPath the YAML file that provides the Sidecar {@link Configuration}
+     * @return an object to read the YAML file from {@code confPath}
+     * @throws IOException when reading the configuration from file fails
+     */
+    private static YAMLConfiguration yamlConfiguration(String confPath) throws IOException
+    {
+        logger.info("Reading configuration from {}", confPath);
+
+        try
+        {
+            URL url = new URL(confPath);
+            YAMLConfiguration yamlConf = new YAMLConfiguration();
+            InputStream stream = url.openStream();
+            yamlConf.read(stream);
+            return yamlConf;
+        }
+        catch (ConfigurationException | IOException e)
+        {
+            throw new IOException(String.format("Unable to parse cluster information from file='%s'", confPath), e);
+        }
+    }
+
+    /**
+     * Parses the {@link InstancesConfig} from the {@link YAMLConfiguration yamlConf}, the {@code versionProvider}, and
+     * the {@code healthCheckFrequencyMillis}.
+     *
+     * @param yamlConf                   the object used to parse the YAML file
+     * @param versionProvider            a Cassandra version provider
+     * @param healthCheckFrequencyMillis the health check frequency configuration in milliseconds
+     * @return the parsed {@link InstancesConfig} from the {@code yamlConf} object
+     */
+    private static InstancesConfig instancesConfig(YAMLConfiguration yamlConf, CassandraVersionProvider versionProvider,
+                                                   long healthCheckFrequencyMillis)
+    {
+        /* Since we are supporting handling multiple instances in Sidecar optionally, we prefer reading single instance
+         * data over reading multiple instances section
+         */
+        org.apache.commons.configuration2.Configuration singleInstanceConf = yamlConf.subset(CASSANDRA_INSTANCE);
+        if (singleInstanceConf != null && !singleInstanceConf.isEmpty())
+        {
+            InstanceMetadata instanceMetadata = buildInstanceMetadata(singleInstanceConf,
+                                                                      versionProvider,
+                                                                      healthCheckFrequencyMillis);
+            return new InstancesConfigImpl(instanceMetadata);
+        }
+
+        List<HierarchicalConfiguration<ImmutableNode>> instances = yamlConf.configurationsAt(CASSANDRA_INSTANCES);
+        final List<InstanceMetadata> instanceMetas = new ArrayList<>();
+        for (HierarchicalConfiguration<ImmutableNode> instance : instances)
+        {
+            InstanceMetadata instanceMetadata = buildInstanceMetadata(instance,
+                                                                      versionProvider,
+                                                                      healthCheckFrequencyMillis);
+            instanceMetas.add(instanceMetadata);
+        }
+        return new InstancesConfigImpl(instanceMetas);
+    }
+
+    /**
+     * Parses the {@link ValidationConfiguration} from the {@link YAMLConfiguration yamlConf}.
+     *
+     * @param yamlConf the object used to parse the YAML file
+     * @return the parsed {@link ValidationConfiguration} from the {@code yamlConf} object
+     */
+    private static ValidationConfiguration validationConfiguration(YAMLConfiguration yamlConf)
+    {
+        org.apache.commons.configuration2.Configuration validation = yamlConf.subset(CASSANDRA_INPUT_VALIDATION);
+        Set<String> forbiddenKeyspaces = new HashSet<>(validation.getList(String.class,
+                                                                          CASSANDRA_FORBIDDEN_KEYSPACES,
+                                                                          Collections.emptyList()));
+        UnaryOperator<String> readString = key -> validation.get(String.class, key);
+        String allowedPatternForDirectory = readString.apply(CASSANDRA_ALLOWED_CHARS_FOR_DIRECTORY);
+        String allowedPatternForComponentName = readString.apply(CASSANDRA_ALLOWED_CHARS_FOR_COMPONENT_NAME);
+        String allowedPatternForRestrictedComponentName = readString
+                                                          .apply(CASSANDRA_ALLOWED_CHARS_FOR_RESTRICTED_COMPONENT_NAME);
+
+        return new YAMLValidationConfiguration(forbiddenKeyspaces,
+                                               allowedPatternForDirectory,
+                                               allowedPatternForComponentName,
+                                               allowedPatternForRestrictedComponentName);
+    }
+
+    /**
+     * Builds the {@link InstanceMetadata} from the {@link org.apache.commons.configuration2.Configuration},
+     * a provided {@code  versionProvider} and {@code healthCheckFrequencyMillis}.
+     *
+     * @param instance                   the object that allows reading from the YAML file
+     * @param versionProvider            a Cassandra version provider
+     * @param healthCheckFrequencyMillis the health check frequency configuration in milliseconds
+     * @return the parsed {@link InstanceMetadata} from YAML
+     */
+    private static InstanceMetadata buildInstanceMetadata(org.apache.commons.configuration2.Configuration instance,
+                                                          CassandraVersionProvider versionProvider,
+                                                          long healthCheckFrequencyMillis)
+    {
+        int id = instance.get(Integer.class, CASSANDRA_INSTANCE_ID, 1);
+        String host = instance.get(String.class, CASSANDRA_INSTANCE_HOST);
+        int port = instance.get(Integer.class, CASSANDRA_INSTANCE_PORT);
+        String dataDirs = instance.get(String.class, CASSANDRA_INSTANCE_DATA_DIRS);
+        String jmxHost = instance.get(String.class, CASSANDRA_JMX_HOST, "127.0.0.1");
+        int jmxPort = instance.get(Integer.class, CASSANDRA_JMX_PORT, 7199);
+        String jmxRole = instance.get(String.class, CASSANDRA_JMX_ROLE, null);
+        String jmxRolePassword = instance.get(String.class, CASSANDRA_JMX_ROLE_PASSWORD, null);
+        boolean jmxSslEnabled = instance.get(Boolean.class, CASSANDRA_JMX_SSL_ENABLED, false);
+
+        CQLSession session = new CQLSession(host, port, healthCheckFrequencyMillis);
+        JmxClient jmxClient = new JmxClient(jmxHost, jmxPort, jmxRole, jmxRolePassword, jmxSslEnabled);
+        return new InstanceMetadataImpl(id,
+                                        Collections.unmodifiableList(Arrays.asList(dataDirs.split(","))),
+                                        session,
+                                        jmxClient,
+                                        versionProvider);
+    }
+}
diff --git a/src/main/java/org/apache/cassandra/sidecar/cluster/InstancesConfigImpl.java b/src/main/java/org/apache/cassandra/sidecar/cluster/InstancesConfigImpl.java
index d067580..ac6df46 100644
--- a/src/main/java/org/apache/cassandra/sidecar/cluster/InstancesConfigImpl.java
+++ b/src/main/java/org/apache/cassandra/sidecar/cluster/InstancesConfigImpl.java
@@ -18,6 +18,7 @@
 
 package org.apache.cassandra.sidecar.cluster;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -33,6 +34,11 @@ public class InstancesConfigImpl implements InstancesConfig
     private final Map<String, InstanceMetadata> hostToInstanceMetas;
     private final List<InstanceMetadata> instanceMetas;
 
+    public InstancesConfigImpl(InstanceMetadata instanceMetadata)
+    {
+        this(Collections.singletonList(instanceMetadata));
+    }
+
     public InstancesConfigImpl(List<InstanceMetadata> instanceMetas)
     {
         this.idToInstanceMetas = instanceMetas.stream()
diff --git a/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadata.java b/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadata.java
index 444e999..1c6a0b8 100644
--- a/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadata.java
+++ b/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadata.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.common.JmxClient;
 
 /**
  * Metadata of an instance
@@ -29,32 +30,37 @@ import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
 public interface InstanceMetadata
 {
     /**
-     * Instance id.
+     * @return an identifier for the Cassandra instance
      */
     int id();
 
     /**
-     * Host address of cassandra instance.
+     * @return the host address of the Cassandra instance
      */
     String host();
 
     /**
-     * Port number of cassandra instance.
+     * @return the port number of the Cassandra instance
      */
     int port();
 
     /**
-     * List of data directories of cassandra instance.
+     * @return a list of data directories of cassandra instance
      */
     List<String> dataDirs();
 
     /**
-     * CQLSession for connecting with instance.
+     * @return {@link CQLSession} for connecting with instance via CQL
      */
     CQLSession session();
 
     /**
-     * Delegate specific for the instance.
+     * @return {@link JmxClient} for connecting with the instance via JMX
+     */
+    JmxClient jmxClient();
+
+    /**
+     * @return a {@link CassandraAdapterDelegate} specific for the instance
      */
     CassandraAdapterDelegate delegate();
 }
diff --git a/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadataImpl.java b/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadataImpl.java
index f63d613..cd34e76 100644
--- a/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadataImpl.java
+++ b/src/main/java/org/apache/cassandra/sidecar/cluster/instance/InstanceMetadataImpl.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.CassandraAdapterDelegate;
 import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
+import org.apache.cassandra.sidecar.common.JmxClient;
 
 /**
  * Local implementation of InstanceMetadata.
@@ -30,49 +31,62 @@ import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
 public class InstanceMetadataImpl implements InstanceMetadata
 {
     private final int id;
-    private final String host;
-    private final int port;
     private final List<String> dataDirs;
     private final CQLSession session;
+    private final JmxClient jmxClient;
     private final CassandraAdapterDelegate delegate;
 
-    public InstanceMetadataImpl(int id, String host, int port, List<String> dataDirs, CQLSession session,
-                                CassandraVersionProvider versionProvider, int healthCheckFrequencyMillis)
+    public InstanceMetadataImpl(int id,
+                                List<String> dataDirs,
+                                CQLSession session,
+                                JmxClient jmxClient,
+                                CassandraVersionProvider versionProvider)
     {
         this.id = id;
-        this.host = host;
-        this.port = port;
         this.dataDirs = dataDirs;
 
-        this.session = new CQLSession(host, port, healthCheckFrequencyMillis);
-        this.delegate = new CassandraAdapterDelegate(versionProvider, session);
+        this.session = session;
+        this.jmxClient = jmxClient;
+        this.delegate = new CassandraAdapterDelegate(versionProvider, session, jmxClient);
     }
 
+    @Override
     public int id()
     {
         return id;
     }
 
+    @Override
     public String host()
     {
-        return host;
+        return session.inet().getHostName();
     }
 
+    @Override
     public int port()
     {
-        return port;
+        return session.inet().getPort();
     }
 
+    @Override
     public List<String> dataDirs()
     {
         return dataDirs;
     }
 
+    @Override
     public CQLSession session()
     {
         return session;
     }
 
+    @Override
+    public JmxClient jmxClient()
+    {
+        return jmxClient;
+    }
+
+    @Override
     public CassandraAdapterDelegate delegate()
     {
         return delegate;
diff --git a/src/main/java/org/apache/cassandra/sidecar/utils/YAMLKeyConstants.java b/src/main/java/org/apache/cassandra/sidecar/utils/YAMLKeyConstants.java
index 2d6d0cf..203c9f0 100644
--- a/src/main/java/org/apache/cassandra/sidecar/utils/YAMLKeyConstants.java
+++ b/src/main/java/org/apache/cassandra/sidecar/utils/YAMLKeyConstants.java
@@ -44,6 +44,13 @@ public class YAMLKeyConstants
     public static final String CASSANDRA_INSTANCE_HOST = "host";
     public static final String CASSANDRA_INSTANCE_PORT = "port";
     public static final String CASSANDRA_INSTANCE_DATA_DIRS = "data_dirs";
+    public static final String CASSANDRA_JMX_HOST = "jmx_host";
+    public static final String CASSANDRA_JMX_PORT = "jmx_port";
+    public static final String CASSANDRA_JMX_ROLE = "jmx_role";
+    public static final String CASSANDRA_JMX_ROLE_PASSWORD = "jmx_role_password";
+    public static final String CASSANDRA_JMX_SSL_ENABLED = "jmx_ssl_enabled";
+
+    // validation for cassandra inputs
     public static final String CASSANDRA_INPUT_VALIDATION = "cassandra_input_validation";
     public static final String CASSANDRA_FORBIDDEN_KEYSPACES = "forbidden_keyspaces";
     public static final String CASSANDRA_ALLOWED_CHARS_FOR_DIRECTORY = "allowed_chars_for_directory";
diff --git a/src/test/java/org/apache/cassandra/sidecar/ConfigurationTest.java b/src/test/java/org/apache/cassandra/sidecar/ConfigurationTest.java
deleted file mode 100644
index d499302..0000000
--- a/src/test/java/org/apache/cassandra/sidecar/ConfigurationTest.java
+++ /dev/null
@@ -1,118 +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;
-
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-
-import org.apache.commons.configuration2.YAMLConfiguration;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.util.Modules;
-import org.apache.cassandra.sidecar.cluster.InstancesConfig;
-import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
-import org.apache.cassandra.sidecar.common.utils.ValidationConfiguration;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-
-/**
- * Test changes related to sidecar.yaml file.
- */
-public class ConfigurationTest
-{
-    private CassandraVersionProvider versionProvider;
-
-    @BeforeEach
-    void setUp()
-    {
-        Injector injector = Guice.createInjector(Modules.override(new MainModule()).with(new TestModule()));
-        versionProvider = injector.getInstance(CassandraVersionProvider.class);
-    }
-
-    @Test
-    public void testOldSidecarYAMLFormatWithSingleInstance() throws Exception
-    {
-        MainModule mainModule = new MainModule();
-        YAMLConfiguration yamlConf = new YAMLConfiguration();
-        try (InputStream stream =
-             Files.newInputStream(Paths.get("src/test/resources/sidecar_single_instance.yaml")))
-        {
-            yamlConf.read(stream);
-            InstancesConfig instancesConfig = mainModule.readInstancesConfig(yamlConf, versionProvider);
-            assertThat(instancesConfig.instances().size()).isEqualTo(1);
-            assertThat(instancesConfig.instances().get(0).host()).isEqualTo("localhost");
-            assertThat(instancesConfig.instances().get(0).port()).isEqualTo(9042);
-        }
-    }
-
-    @Test
-    public void testReadingSingleInstanceSectionOverMultipleInstances() throws Exception
-    {
-        MainModule mainModule = new MainModule();
-        YAMLConfiguration yamlConf = new YAMLConfiguration();
-        try (InputStream stream =
-             Files.newInputStream(Paths.get("src/test/resources/sidecar_with_single_multiple_instances.yaml")))
-        {
-            yamlConf.read(stream);
-            InstancesConfig instancesConfig = mainModule.readInstancesConfig(yamlConf, versionProvider);
-            assertThat(instancesConfig.instances().size()).isEqualTo(1);
-            assertThat(instancesConfig.instances().get(0).host()).isEqualTo("localhost");
-            assertThat(instancesConfig.instances().get(0).port()).isEqualTo(9042);
-        }
-    }
-
-    @Test
-    public void testReadingMultipleInstances() throws Exception
-    {
-        MainModule mainModule = new MainModule();
-        YAMLConfiguration yamlConf = new YAMLConfiguration();
-        try (InputStream stream =
-             Files.newInputStream(Paths.get("src/test/resources/sidecar_multiple_instances.yaml")))
-        {
-            yamlConf.read(stream);
-            InstancesConfig instancesConfig = mainModule.readInstancesConfig(yamlConf, versionProvider);
-            assertThat(instancesConfig.instances().size()).isEqualTo(2);
-        }
-    }
-
-    @Test
-    public void testReadingCassandraInputValidation() throws Exception
-    {
-        MainModule mainModule = new MainModule();
-        YAMLConfiguration yamlConf = new YAMLConfiguration();
-        try (InputStream stream =
-             Files.newInputStream(Paths.get("src/test/resources/sidecar_validation_configuration.yaml")))
-        {
-            yamlConf.read(stream);
-            ValidationConfiguration validationConfiguration = mainModule.validationConfiguration(yamlConf);
-
-            assertThat(validationConfiguration.getForbiddenKeyspaces()).contains("a", "b", "c");
-            assertThat(validationConfiguration.getAllowedPatternForDirectory()).isEqualTo("[a-z]+");
-            assertThat(validationConfiguration.getAllowedPatternForComponentName())
-            .isEqualTo("(.db|.cql|.json|.crc32|TOC.txt)");
-            assertThat(validationConfiguration.getAllowedPatternForRestrictedComponentName())
-            .isEqualTo("(.db|TOC.txt)");
-        }
-    }
-}
diff --git a/src/test/java/org/apache/cassandra/sidecar/YAMLSidecarConfigurationTest.java b/src/test/java/org/apache/cassandra/sidecar/YAMLSidecarConfigurationTest.java
new file mode 100644
index 0000000..0a621a1
--- /dev/null
+++ b/src/test/java/org/apache/cassandra/sidecar/YAMLSidecarConfigurationTest.java
@@ -0,0 +1,84 @@
+package org.apache.cassandra.sidecar;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Objects;
+
+import org.junit.jupiter.api.Test;
+
+import org.apache.cassandra.sidecar.cluster.InstancesConfig;
+import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
+import org.apache.cassandra.sidecar.common.utils.ValidationConfiguration;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests reading Sidecar {@link Configuration} from YAML files
+ */
+class YAMLSidecarConfigurationTest
+{
+    CassandraVersionProvider versionProvider = mock(CassandraVersionProvider.class);
+
+    @Test
+    public void testLegacySidecarYAMLFormatWithSingleInstance() throws IOException
+    {
+        Configuration configuration =
+        YAMLSidecarConfiguration.of(confPath("sidecar_single_instance.yaml"), versionProvider);
+        InstancesConfig instancesConfig = configuration.getInstancesConfig();
+        assertThat(instancesConfig.instances().size()).isEqualTo(1);
+        InstanceMetadata instanceMetadata = instancesConfig.instances().get(0);
+        assertThat(instanceMetadata.host()).isEqualTo("localhost");
+        assertThat(instanceMetadata.port()).isEqualTo(9042);
+        assertThat(instanceMetadata.session()).isNotNull();
+        assertThat(instanceMetadata.jmxClient()).isNotNull();
+    }
+
+    @Test
+    public void testReadingSingleInstanceSectionOverMultipleInstances() throws IOException
+    {
+        Configuration configuration =
+        YAMLSidecarConfiguration.of(confPath("sidecar_with_single_multiple_instances.yaml"),
+                                    versionProvider);
+        InstancesConfig instancesConfig = configuration.getInstancesConfig();
+        assertThat(instancesConfig.instances().size()).isEqualTo(1);
+        InstanceMetadata instanceMetadata = instancesConfig.instances().get(0);
+        assertThat(instanceMetadata.host()).isEqualTo("localhost");
+        assertThat(instanceMetadata.port()).isEqualTo(9042);
+        assertThat(instanceMetadata.session()).isNotNull();
+        assertThat(instanceMetadata.jmxClient()).isNotNull();
+    }
+
+    @Test
+    public void testReadingMultipleInstances() throws IOException
+    {
+        Configuration configuration =
+        YAMLSidecarConfiguration.of(confPath("sidecar_multiple_instances.yaml"),
+                                    versionProvider);
+        InstancesConfig instancesConfig = configuration.getInstancesConfig();
+        assertThat(instancesConfig.instances().size()).isEqualTo(2);
+    }
+
+    @Test
+    public void testReadingCassandraInputValidation() throws IOException
+    {
+        Configuration configuration =
+        YAMLSidecarConfiguration.of(confPath("sidecar_validation_configuration.yaml"),
+                                    versionProvider);
+        ValidationConfiguration validationConfiguration = configuration.getValidationConfiguration();
+
+        assertThat(validationConfiguration.getForbiddenKeyspaces()).contains("a", "b", "c");
+        assertThat(validationConfiguration.getAllowedPatternForDirectory()).isEqualTo("[a-z]+");
+        assertThat(validationConfiguration.getAllowedPatternForComponentName())
+        .isEqualTo("(.db|.cql|.json|.crc32|TOC.txt)");
+        assertThat(validationConfiguration.getAllowedPatternForRestrictedComponentName())
+        .isEqualTo("(.db|TOC.txt)");
+    }
+
+    private String confPath(String resourceName)
+    {
+        URL resource = getClass().getClassLoader().getResource(resourceName);
+        return Objects.requireNonNull(resource).toString();
+    }
+}
diff --git a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java
index d482fbd..eb46e4a 100644
--- a/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java
+++ b/src/test/java/org/apache/cassandra/sidecar/snapshots/SnapshotUtils.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.sidecar.snapshots;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,17 +33,19 @@ import org.apache.cassandra.sidecar.cluster.InstancesConfig;
 import org.apache.cassandra.sidecar.cluster.InstancesConfigImpl;
 import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
 import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadataImpl;
+import org.apache.cassandra.sidecar.common.CQLSession;
 import org.apache.cassandra.sidecar.common.CassandraVersionProvider;
 import org.apache.cassandra.sidecar.common.MockCassandraFactory;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Utilities for testing snapshot related features
  */
 public class SnapshotUtils
 {
-    @SuppressWarnings("ResultOfMethodCallIgnored")
     public static void initializeTmpDirectory(File temporaryFolder) throws IOException
     {
         for (final String[] folderPath : getMockSnapshotDirectories())
@@ -78,20 +81,20 @@ public class SnapshotUtils
         CassandraVersionProvider.Builder versionProviderBuilder = new CassandraVersionProvider.Builder();
         versionProviderBuilder.add(new MockCassandraFactory());
         CassandraVersionProvider versionProvider = versionProviderBuilder.build();
+        CQLSession mockSession1 = mock(CQLSession.class);
+        when(mockSession1.inet()).thenReturn(InetSocketAddress.createUnresolved("localhost", 9043));
+        CQLSession mockSession2 = mock(CQLSession.class);
+        when(mockSession2.inet()).thenReturn(InetSocketAddress.createUnresolved("localhost2", 9043));
         InstanceMetadataImpl localhost = new InstanceMetadataImpl(1,
-                                                                  "localhost",
-                                                                  9043,
                                                                   Collections.singletonList(rootPath + "/d1"),
+                                                                  mockSession1,
                                                                   null,
-                                                                  versionProvider,
-                                                                  1000);
+                                                                  versionProvider);
         InstanceMetadataImpl localhost2 = new InstanceMetadataImpl(2,
-                                                                   "localhost2",
-                                                                   9043,
                                                                    Collections.singletonList(rootPath + "/d2"),
+                                                                   mockSession2,
                                                                    null,
-                                                                   versionProvider,
-                                                                   1000);
+                                                                   versionProvider);
         List<InstanceMetadata> instanceMetas = Arrays.asList(localhost, localhost2);
         return new InstancesConfigImpl(instanceMetas);
     }
diff --git a/src/test/resources/sidecar_multiple_instances.yaml b/src/test/resources/sidecar_multiple_instances.yaml
index 5b8d280..dea05b0 100644
--- a/src/test/resources/sidecar_multiple_instances.yaml
+++ b/src/test/resources/sidecar_multiple_instances.yaml
@@ -6,10 +6,20 @@ cassandra_instances:
     host: localhost1
     port: 9042
     data_dirs: /cassandra/d1/data, /cassandra/d2/data
+    jmx_host: 127.0.0.1
+    jmx_port: 7100
+    jmx_ssl_enabled: false
+  #    jmx_role:
+  #    jmx_role_password:
   - id: 2
     host: localhost2
     port: 9042
     data_dirs: /cassandra/d3/data, /cassandra/d4/data
+    jmx_host: 127.0.0.1
+    jmx_port: 7200
+    jmx_ssl_enabled: false
+#    jmx_role:
+#    jmx_role_password:
 
 sidecar:
   - host: 0.0.0.0
@@ -32,4 +42,4 @@ sidecar:
 
 
 healthcheck:
-  - poll_freq_millis: 30000
\ No newline at end of file
+  - poll_freq_millis: 30000
diff --git a/src/test/resources/sidecar_single_instance.yaml b/src/test/resources/sidecar_single_instance.yaml
index a3823a6..9bbc301 100644
--- a/src/test/resources/sidecar_single_instance.yaml
+++ b/src/test/resources/sidecar_single_instance.yaml
@@ -5,6 +5,11 @@ cassandra:
   - host: localhost
   - port: 9042
   - data_dirs: /cassandra/d1/data, /cassandra/d2/data
+  - jmx_host: 127.0.0.1
+  - jmx_port: 7199
+  - jmx_role: controlRole
+  - jmx_role_password: controlPassword
+  - jmx_ssl_enabled: true
 
 sidecar:
   - host: 0.0.0.0
diff --git a/src/test/resources/sidecar_with_single_multiple_instances.yaml b/src/test/resources/sidecar_with_single_multiple_instances.yaml
index 3486886..51872c8 100644
--- a/src/test/resources/sidecar_with_single_multiple_instances.yaml
+++ b/src/test/resources/sidecar_with_single_multiple_instances.yaml
@@ -5,16 +5,31 @@ cassandra:
   - host: localhost
   - port: 9042
   - data_dirs: /cassandra/d1/data, /cassandra/d2/data
+  - jmx_host: 127.0.0.1
+  - jmx_port: 7199
+  - jmx_role: controlRole
+  - jmx_role_password: controlPassword
+  - jmx_ssl_enabled: true
 
 cassandra_instances:
   - id: 1
     host: localhost1
     port: 9042
     data_dirs: /cassandra/d1/data, /cassandra/d2/data
+    jmx_host: 127.0.0.1
+    jmx_port: 7100
+    jmx_role: controlRole
+    jmx_role_password: controlPassword
+    jmx_ssl_enabled: true
   - id: 2
     host: localhost2
     port: 9042
     data_dirs: /cassandra/d3/data, /cassandra/d4/data
+    jmx_host: 127.0.0.1
+    jmx_port: 7200
+    jmx_role: controlRole
+    jmx_role_password: controlPassword
+    jmx_ssl_enabled: true
 
 sidecar:
   - host: 0.0.0.0
@@ -37,4 +52,4 @@ sidecar:
 
 
 healthcheck:
-  - poll_freq_millis: 30000
\ No newline at end of file
+  - poll_freq_millis: 30000


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