CASSANDRASC-63: Support credential rotation in JmxClient

JMX credentials in a Cassandra instance can be rotated on a cadence, on every bounce, or by some other
means. In those cases, the `JmxClient` will no longer be able to connect to the instance completely
losing the ability to talk to that instance.

In this commit, we allow the `JmxClient` to support credential changes to be continue to talk to the
Cassandra instance uninterrupted without any potential downtime to the Sidecar service.

patch by Francisco Guerrero; reviewed by Dinesh Joshi, Yifan Cai for CASSANDRASC-63

# Conflicts:
#	CHANGES.txt
diff --git a/CHANGES.txt b/CHANGES.txt
index 5dc930b..8e5826f 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 1.0.0
 -----
+ * Support credential rotation in JmxClient (CASSANDRASC-63)
  * File descriptor is not being closed on MD5 checksum (CASSANDRASC-64)
  * Expose JMX host and port from JMXClient (CASSANDRASC-59)
  * Support retries in Sidecar Client on Invalid Checksum (CASSANDRASC-58)
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java b/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
index b5db0a5..d5cdcf3 100644
--- a/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/JmxClient.java
@@ -25,6 +25,9 @@
 import java.rmi.server.RMISocketFactory;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
+import java.util.function.BooleanSupplier;
+import java.util.function.Supplier;
 import javax.management.JMX;
 import javax.management.MBeanServerConnection;
 import javax.management.MalformedObjectNameException;
@@ -37,6 +40,7 @@
 import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
+import org.apache.cassandra.sidecar.common.exceptions.JmxAuthenticationException;
 import org.jetbrains.annotations.VisibleForTesting;
 
 /**
@@ -49,9 +53,11 @@
     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;
     private JMXConnector jmxConnector;
+    private final Supplier<String> roleSupplier;
+    private final Supplier<String> passwordSupplier;
+    private final BooleanSupplier enableSslSupplier;
 
     /**
      * Creates a new client with the provided {@code host} and {@code port}.
@@ -75,32 +81,30 @@
      */
     public JmxClient(String host, int port, String role, String password, boolean enableSSl)
     {
-        this(buildJmxServiceURL(host, port), role, password, enableSSl);
+        this(buildJmxServiceURL(host, port), () -> role, () -> password, () -> enableSSl);
     }
 
     @VisibleForTesting
     JmxClient(JMXServiceURL jmxServiceURL)
     {
-        this(jmxServiceURL, null, null, false);
+        this(jmxServiceURL, () -> null, () -> null, () -> false);
     }
 
     @VisibleForTesting
     JmxClient(JMXServiceURL jmxServiceURL, String role, String password)
     {
-        this(jmxServiceURL, role, password, false);
+        this(jmxServiceURL, () -> role, () -> password, () -> false);
     }
 
-    private JmxClient(JMXServiceURL jmxServiceURL, String role, String password, boolean enableSsl)
+    public JmxClient(JMXServiceURL jmxServiceURL,
+                     Supplier<String> roleSupplier,
+                     Supplier<String> passwordSupplier,
+                     BooleanSupplier enableSslSupplier)
     {
-        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, rmiClientSocketFactory(enableSsl));
+        this.jmxServiceURL = Objects.requireNonNull(jmxServiceURL, "jmxServiceURL is required");
+        this.roleSupplier = Objects.requireNonNull(roleSupplier, "roleSupplier is required");
+        this.passwordSupplier = Objects.requireNonNull(passwordSupplier, "passwordSupplier is required");
+        this.enableSslSupplier = Objects.requireNonNull(enableSslSupplier, "enableSslSupplier is required");
     }
 
     /**
@@ -147,7 +151,7 @@
     {
         try
         {
-            jmxConnector = JMXConnectorFactory.connect(jmxServiceURL, jmxEnv);
+            jmxConnector = JMXConnectorFactory.connect(jmxServiceURL, buildJmxEnv());
             jmxConnector.addConnectionNotificationListener(this, null, null);
             mBeanServerConnection = jmxConnector.getMBeanServerConnection();
             connected = true;
@@ -158,6 +162,19 @@
             throw new RuntimeException(String.format("Failed to connect to JMX endpoint %s", jmxServiceURL),
                                        iox);
         }
+        catch (SecurityException securityException)
+        {
+            connected = false;
+            String errorMessage = securityException.getMessage() != null
+                                  ? securityException.getMessage()
+                                  : "JMX Authentication failed";
+            throw new JmxAuthenticationException(errorMessage, securityException);
+        }
+        catch (RuntimeException runtimeException)
+        {
+            // catch exceptions coming from the lambdas and wrap them in a JmxAuthenticationException
+            throw new JmxAuthenticationException(runtimeException);
+        }
     }
 
     @Override
@@ -216,6 +233,22 @@
         }
     }
 
+    private Map<String, Object> buildJmxEnv()
+    {
+        String role = roleSupplier.get();
+        String password = passwordSupplier.get();
+        boolean enableSsl = enableSslSupplier.getAsBoolean();
+
+        Map<String, Object> 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, rmiClientSocketFactory(enableSsl));
+        return jmxEnv;
+    }
+
     @Override
     public synchronized void close() throws IOException
     {
diff --git a/common/src/main/java/org/apache/cassandra/sidecar/common/exceptions/JmxAuthenticationException.java b/common/src/main/java/org/apache/cassandra/sidecar/common/exceptions/JmxAuthenticationException.java
new file mode 100644
index 0000000..f09f778
--- /dev/null
+++ b/common/src/main/java/org/apache/cassandra/sidecar/common/exceptions/JmxAuthenticationException.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.common.exceptions;
+
+/**
+ * Exceptions encountered during JMX client authentication to the server
+ */
+public class JmxAuthenticationException extends RuntimeException
+{
+    /**
+     * Constructs a JMX authentication exception with the specified detail
+     * message.
+     *
+     * @param message the detail message (which is saved for later retrieval
+     *                by the {@link #getMessage()} method).
+     */
+    public JmxAuthenticationException(String message)
+    {
+        super(message);
+    }
+
+    /**
+     * Constructs a JMX authentication exception with the specified cause
+     * and a detail message of {@code (cause==null ? null : cause.toString())}
+     * (which typically contains the class and detail message of
+     * {@code cause}).  This constructor is useful for runtime exceptions
+     * that are little more than wrappers for other throwables.
+     *
+     * @param cause the cause (which is saved for later retrieval by the
+     *              {@link #getCause()} method).  (A {@code null} value is
+     *              permitted, and indicates that the cause is nonexistent or
+     *              unknown.)
+     */
+    public JmxAuthenticationException(Throwable cause)
+    {
+        super(cause);
+    }
+
+    /**
+     * Constructs a JMX authentication exception with the specified detail
+     * message and cause.
+     *
+     * @param message the detail message (which is saved for later retrieval
+     *                by the {@link #getMessage()} method).
+     * @param cause   the cause (which is saved for later retrieval by the
+     *                {@link #getCause()} method).  (A {@code null} value is
+     *                permitted, and indicates that the cause is nonexistent or
+     *                unknown.)
+     */
+    public JmxAuthenticationException(String message, Throwable cause)
+    {
+        super(message, cause);
+    }
+}
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
index 4b716f6..e3269f6 100644
--- a/common/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java
+++ b/common/src/test/java/org/apache/cassandra/sidecar/common/JmxClientTest.java
@@ -32,6 +32,9 @@
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BooleanSupplier;
+import java.util.function.Supplier;
 import java.util.logging.Logger;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
@@ -46,6 +49,8 @@
 import org.junit.jupiter.api.Test;
 import org.junit.platform.commons.util.Preconditions;
 
+import org.apache.cassandra.sidecar.common.exceptions.JmxAuthenticationException;
+
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
 
@@ -138,23 +143,110 @@
     }
 
     @Test
-    public void testCallWithoutCredentialsFails()
+    public void testCallWithoutCredentialsFails() throws IOException
     {
-        assertThatExceptionOfType(SecurityException.class)
-        .isThrownBy(() ->
-                    {
-                        try (JmxClient client = new JmxClient(serviceURL))
-                        {
-                            client.proxy(Import.class, objectName)
-                                  .importNewSSTables(Sets.newHashSet("foo", "bar"),
-                                                     true,
-                                                     true,
-                                                     true,
-                                                     true,
-                                                     true,
-                                                     true);
-                        }
-                    });
+        try (JmxClient client = new JmxClient(serviceURL))
+        {
+            assertThatExceptionOfType(JmxAuthenticationException.class)
+            .isThrownBy(() ->
+                        client.proxy(Import.class, objectName)
+                              .importNewSSTables(Sets.newHashSet("foo", "bar"),
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true))
+            .withMessageContaining("Authentication failed! Credentials required");
+        }
+    }
+
+    @Test
+    public void testRoleSupplierThrows() throws IOException
+    {
+        String errorMessage = "bad role state!";
+        Supplier<String> roleSupplier = () -> {
+            throw new IllegalStateException(errorMessage);
+        };
+        testSupplierThrows(errorMessage,
+                           new JmxClient(serviceURL, roleSupplier, () -> "password", () -> false));
+    }
+
+    @Test
+    public void testPasswordSupplierThrows() throws IOException
+    {
+        String errorMessage = "bad password state!";
+        Supplier<String> passwordSupplier = () -> {
+            throw new IllegalStateException(errorMessage);
+        };
+        testSupplierThrows(errorMessage,
+                           new JmxClient(serviceURL, () -> "controlRole", passwordSupplier, () -> false));
+    }
+
+    @Test
+    public void testEnableSslSupplierThrows() throws IOException
+    {
+        String errorMessage = "bad ssl supplier state!";
+        BooleanSupplier enableSslSupplier = () -> {
+            throw new IllegalStateException(errorMessage);
+        };
+        testSupplierThrows(errorMessage,
+                           new JmxClient(serviceURL, () -> "controlRole", () -> "password", enableSslSupplier));
+    }
+
+    private static void testSupplierThrows(String errorMessage, JmxClient jmxClient) throws IOException
+    {
+        try (JmxClient client = jmxClient)
+        {
+            assertThatExceptionOfType(JmxAuthenticationException.class)
+            .isThrownBy(() ->
+                        client.proxy(Import.class, objectName)
+                              .importNewSSTables(Sets.newHashSet("foo", "bar"),
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true))
+            .withMessageContaining(errorMessage);
+        }
+    }
+
+    @Test
+    public void testRetryAfterAuthenticationFailureWithCorrectCredentials() throws IOException
+    {
+        AtomicInteger tryCount = new AtomicInteger(0);
+        List<String> result;
+        Supplier<String> passwordSupplier = () -> {
+            if (tryCount.getAndIncrement() == 0)
+            {
+                // authentication fails on the first attempt
+                return "bad password";
+            }
+            return "password";
+        };
+        try (JmxClient client = new JmxClient(serviceURL, () -> "controlRole", passwordSupplier, () -> false))
+        {
+            // First attempt fails
+            assertThatExceptionOfType(JmxAuthenticationException.class)
+            .isThrownBy(() ->
+                        client.proxy(Import.class, objectName)
+                              .importNewSSTables(Sets.newHashSet("foo", "bar"),
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true,
+                                                 true))
+            .withMessageContaining("Authentication failed! Invalid username or password");
+
+            // second attempt succeeds after getting the correct password
+            result = client.proxy(Import.class, objectName)
+                           .importNewSSTables(Sets.newHashSet("foo", "bar"), true,
+                                              true, true, true, true,
+                                              true);
+        }
+        assertThat(result.size()).isEqualTo(0);
     }
 
     @Test
diff --git a/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java b/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
index 78c8b24..12cd48a 100644
--- a/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
+++ b/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java
@@ -32,6 +32,7 @@
 import io.vertx.ext.web.handler.HttpException;
 import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
 import org.apache.cassandra.sidecar.common.data.QualifiedTableName;
+import org.apache.cassandra.sidecar.common.exceptions.JmxAuthenticationException;
 import org.apache.cassandra.sidecar.common.utils.CassandraInputValidator;
 import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
 import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
@@ -165,9 +166,7 @@
     protected void processFailure(Throwable cause, RoutingContext context, String host, SocketAddress remoteAddress,
                                   T request)
     {
-        HttpException httpException = cause instanceof HttpException
-                                      ? (HttpException) cause
-                                      : wrapHttpException(HttpResponseStatus.INTERNAL_SERVER_ERROR, cause);
+        HttpException httpException = determineHttpException(cause);
 
         if (HttpStatusClass.CLIENT_ERROR.contains(httpException.getStatusCode()))
         {
@@ -183,6 +182,21 @@
         context.fail(httpException);
     }
 
+    protected HttpException determineHttpException(Throwable cause)
+    {
+        if (cause instanceof HttpException)
+        {
+            return (HttpException) cause;
+        }
+
+        if (cause instanceof JmxAuthenticationException)
+        {
+            return wrapHttpException(HttpResponseStatus.SERVICE_UNAVAILABLE, cause);
+        }
+
+        return wrapHttpException(HttpResponseStatus.INTERNAL_SERVER_ERROR, cause);
+    }
+
     /**
      * Returns the validated {@link QualifiedTableName} from the context, where the both keyspace and table name
      * are required.
diff --git a/src/test/java/org/apache/cassandra/sidecar/TestModule.java b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
index 1573972..4723944 100644
--- a/src/test/java/org/apache/cassandra/sidecar/TestModule.java
+++ b/src/test/java/org/apache/cassandra/sidecar/TestModule.java
@@ -64,7 +64,7 @@
     protected Configuration abstractConfig(InstancesConfig instancesConfig)
     {
         WorkerPoolConfiguration workPoolConf = new WorkerPoolConfiguration("test-pool", 10, 30000);
-        return new Configuration.Builder()
+        return new Configuration.Builder<>()
                .setInstancesConfig(instancesConfig)
                .setHost("127.0.0.1")
                .setPort(6475)
diff --git a/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java b/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java
index 3ed4e7e..5e39e83 100644
--- a/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java
+++ b/src/test/java/org/apache/cassandra/sidecar/TestSslModule.java
@@ -18,7 +18,8 @@
 
 package org.apache.cassandra.sidecar;
 
-import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,46 +35,49 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(TestSslModule.class);
 
-
     @Override
     public Configuration abstractConfig(InstancesConfig instancesConfig)
     {
-        final String keyStorePath = TestSslModule.class.getClassLoader().getResource("certs/test.p12").getPath();
-        final String keyStorePassword = "password";
+        String keyStorePath = TestSslModule.class.getClassLoader()
+                                                 .getResource("certs/test.p12")
+                                                 .getPath();
+        String keyStorePassword = "password";
 
-        final String trustStorePath = TestSslModule.class.getClassLoader().getResource("certs/ca.p12").getPath();
-        final String trustStorePassword = "password";
+        String trustStorePath = TestSslModule.class.getClassLoader()
+                                                         .getResource("certs/ca.p12")
+                                                         .getPath();
+        String trustStorePassword = "password";
 
-        if (!new File(keyStorePath).exists())
+        if (!Files.exists(Paths.get(keyStorePath)))
         {
-            logger.error("JMX password file not found");
+            logger.error("JMX password file not found in path={}", keyStorePath);
         }
-        if (!new File(trustStorePath).exists())
+        if (!Files.exists(Paths.get(trustStorePath)))
         {
-            logger.error("Trust Store file not found");
+            logger.error("Trust Store file not found in path={}", trustStorePath);
         }
 
         WorkerPoolConfiguration workerPoolConf = new WorkerPoolConfiguration("test-pool", 10,
                                                                              30000);
 
-        return new Configuration.Builder()
-                           .setInstancesConfig(instancesConfig)
-                           .setHost("127.0.0.1")
-                           .setPort(6475)
-                           .setHealthCheckFrequency(1000)
-                           .setKeyStorePath(keyStorePath)
-                           .setKeyStorePassword(keyStorePassword)
-                           .setTrustStorePath(trustStorePath)
-                           .setTrustStorePassword(trustStorePassword)
-                           .setSslEnabled(true)
-                           .setRateLimitStreamRequestsPerSecond(1)
-                           .setRequestIdleTimeoutMillis(300_000)
-                           .setRequestTimeoutMillis(300_000L)
-                           .setConcurrentUploadsLimit(80)
-                           .setMinSpacePercentRequiredForUploads(0)
-                           .setSSTableImportCacheConfiguration(new CacheConfiguration(60_000, 100))
-                           .setServerWorkerPoolConfiguration(workerPoolConf)
-                           .setServerInternalWorkerPoolConfiguration(workerPoolConf)
-                           .build();
+        return new Configuration.Builder<>()
+               .setInstancesConfig(instancesConfig)
+               .setHost("127.0.0.1")
+               .setPort(6475)
+               .setHealthCheckFrequency(1000)
+               .setKeyStorePath(keyStorePath)
+               .setKeyStorePassword(keyStorePassword)
+               .setTrustStorePath(trustStorePath)
+               .setTrustStorePassword(trustStorePassword)
+               .setSslEnabled(true)
+               .setRateLimitStreamRequestsPerSecond(1)
+               .setRequestIdleTimeoutMillis(300_000)
+               .setRequestTimeoutMillis(300_000L)
+               .setConcurrentUploadsLimit(80)
+               .setMinSpacePercentRequiredForUploads(0)
+               .setSSTableImportCacheConfiguration(new CacheConfiguration(60_000, 100))
+               .setServerWorkerPoolConfiguration(workerPoolConf)
+               .setServerInternalWorkerPoolConfiguration(workerPoolConf)
+               .build();
     }
 }
diff --git a/src/test/java/org/apache/cassandra/sidecar/routes/RingHandlerTest.java b/src/test/java/org/apache/cassandra/sidecar/routes/RingHandlerTest.java
index d0467dd..73631e8 100644
--- a/src/test/java/org/apache/cassandra/sidecar/routes/RingHandlerTest.java
+++ b/src/test/java/org/apache/cassandra/sidecar/routes/RingHandlerTest.java
@@ -47,7 +47,6 @@
 import io.vertx.ext.web.client.predicate.ResponsePredicate;
 import io.vertx.junit5.VertxExtension;
 import io.vertx.junit5.VertxTestContext;
-import org.apache.cassandra.sidecar.Configuration;
 import org.apache.cassandra.sidecar.MainModule;
 import org.apache.cassandra.sidecar.TestModule;
 import org.apache.cassandra.sidecar.cluster.InstancesConfig;
@@ -56,6 +55,7 @@
 import org.apache.cassandra.sidecar.common.StorageOperations;
 import org.apache.cassandra.sidecar.common.data.RingEntry;
 import org.apache.cassandra.sidecar.common.data.RingResponse;
+import org.apache.cassandra.sidecar.common.exceptions.JmxAuthenticationException;
 import org.mockito.stubbing.Answer;
 
 import static io.netty.handler.codec.http.HttpResponseStatus.OK;
@@ -72,7 +72,6 @@
 {
     static final Logger LOGGER = LoggerFactory.getLogger(RingHandlerTest.class);
     Vertx vertx;
-    Configuration config;
     HttpServer server;
 
     @BeforeEach
@@ -84,9 +83,8 @@
                                                         .with(testOverride));
         vertx = injector.getInstance(Vertx.class);
         server = injector.getInstance(HttpServer.class);
-        config = injector.getInstance(Configuration.class);
         VertxTestContext context = new VertxTestContext();
-        server.listen(config.getPort(), config.getHost(), context.succeedingThenComplete());
+        server.listen(0, "127.0.0.1", context.succeedingThenComplete());
         context.awaitCompletion(5, TimeUnit.SECONDS);
     }
 
@@ -109,7 +107,7 @@
         VertxTestContext context = new VertxTestContext();
         WebClient client = WebClient.create(vertx);
         String testRoute = "/api/v1/cassandra/ring/keyspaces/" + keyspace;
-        client.get(config.getPort(), config.getHost(), testRoute)
+        client.get(server.actualPort(), "127.0.0.1", testRoute)
               .expect(ResponsePredicate.SC_BAD_REQUEST)
               .send(context.succeedingThenComplete());
     }
@@ -135,7 +133,7 @@
         };
         WebClient client = WebClient.create(vertx);
         String testRoute = "/api/v1/cassandra/ring";
-        client.get(config.getPort(), config.getHost(), testRoute)
+        client.get(server.actualPort(), "127.0.0.1", testRoute)
               .expect(ResponsePredicate.SC_OK)
               .send(context.succeeding(response -> {
                   assertThat(response.statusCode()).isEqualTo(OK.code());
@@ -168,7 +166,7 @@
         };
         WebClient client = WebClient.create(vertx);
         String testRoute = "/api/v1/cassandra/ring";
-        client.get(config.getPort(), config.getHost(), testRoute)
+        client.get(server.actualPort(), "127.0.0.1", testRoute)
               .expect(ResponsePredicate.SC_INTERNAL_SERVER_ERROR)
               .send(context.succeeding(response -> {
                   JsonObject error = response.bodyAsJsonObject();
@@ -179,6 +177,27 @@
               }));
     }
 
+    @Test
+    void testGetRingFailsOnUnavailableJmxConnectivity(VertxTestContext context)
+    {
+        String errorMessage = "Authentication failed! Invalid username or password";
+        RingHandlerTestModule.ringSupplier = () -> {
+            throw new JmxAuthenticationException(errorMessage);
+        };
+        WebClient client = WebClient.create(vertx);
+        String testRoute = "/api/v1/cassandra/ring";
+        client.get(server.actualPort(), "127.0.0.1", testRoute)
+              .expect(ResponsePredicate.SC_SERVICE_UNAVAILABLE)
+              .send(context.succeeding(response -> {
+                  JsonObject error = response.bodyAsJsonObject();
+                  assertThat(error.getInteger("code")).isEqualTo(503);
+                  assertThat(error.getString("status")).isEqualTo("Service Unavailable");
+                  assertThat(error.getString("message").contains(errorMessage));
+
+                  context.completeNow();
+              }));
+    }
+
     static class RingHandlerTestModule extends AbstractModule
     {
         static Supplier<RingResponse> ringSupplier;