You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by on...@apache.org on 2021/07/12 16:13:54 UTC

[geode] branch support/1.13 updated: GEODE-9156: Replace docker-compose-rule with testcontainers (#6691)

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

onichols pushed a commit to branch support/1.13
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/support/1.13 by this push:
     new 22ac876  GEODE-9156: Replace docker-compose-rule with testcontainers (#6691)
22ac876 is described below

commit 22ac8760e9afad1cbec3e46e44afdba2f8c0d3c8
Author: Owen Nichols <34...@users.noreply.github.com>
AuthorDate: Mon Jul 12 08:43:22 2021 -0700

    GEODE-9156: Replace docker-compose-rule with testcontainers (#6691)
    
    * GEODE-9156: Replace docker-compose-rule with testcontainers in geode-connectors (#6378) (#6385)
    
    Something to note when doing SSL testing: testcontainers does not let
    you set the container name (using `container_name` in your compose.yml).
    This ultimately means that reverse IP lookups produce hostnames that
    look something like `project_service_index`. The problem is that these
    names are not RFC compliant as they contain underscores. This can break
    some aspects of SSL validation. I've had to work around this by renaming
    containers in various test classes.
    
    (cherry picked from commit a5335756a5801adf35ffdf4635b51cb17757eb84)
    (cherry picked from commit 473af500ce43a4d35e08d4d750f3b5ed9186cc99)
    (cherry picked from commit f537a5dded32d5386dccbb99a7e862b29811e269)
    
    Co-authored-by: Jens Deppe <jd...@vmware.com>
---
 .../src/test/resources/expected-pom.xml            |  12 -
 .../gradle/plugins/DependencyConstraints.groovy    |   5 -
 geode-assembly/build.gradle                        |   7 +-
 .../client/sni/ClientSNICQAcceptanceTest.java      |  57 ++---
 .../sni/ClientSNIDropProxyAcceptanceTest.java      |  82 ++-----
 .../client/sni/DualServerSNIAcceptanceTest.java    |  32 +--
 .../geode/client/sni/NotOnWindowsDockerRule.java   |  57 -----
 .../client/sni/SingleServerSNIAcceptanceTest.java  |  27 +--
 .../org/apache/geode/rules/DockerComposeRule.java  | 268 +++++++++++++++++++++
 .../org/apache/geode/client/sni/docker-compose.yml |   7 -
 geode-connectors/build.gradle                      |  10 +-
 .../jdbc/JdbcAsyncWriterIntegrationTest.java       |   6 +-
 .../geode/connectors/jdbc/JdbcDistributedTest.java |  91 ++++---
 .../connectors/jdbc/JdbcLoaderIntegrationTest.java |   6 +-
 .../connectors/jdbc/JdbcWriterIntegrationTest.java |   6 +-
 .../jdbc/MySqlJdbcAsyncWriterIntegrationTest.java  |  11 +-
 .../connectors/jdbc/MySqlJdbcDistributedTest.java  |  40 +--
 .../jdbc/MySqlJdbcLoaderIntegrationTest.java       |  11 +-
 .../jdbc/MySqlJdbcWriterIntegrationTest.java       |  11 +-
 .../PostgresJdbcAsyncWriterIntegrationTest.java    |  11 +-
 .../jdbc/PostgresJdbcDistributedTest.java          |  40 ++-
 .../jdbc/PostgresJdbcLoaderIntegrationTest.java    |  22 +-
 .../jdbc/PostgresJdbcWriterIntegrationTest.java    |  29 ++-
 .../connectors/jdbc/TestDataSourceFactory.java     |   8 +-
 .../MySqlTableMetaDataManagerIntegrationTest.java  |   6 +-
 ...ostgresTableMetaDataManagerIntegrationTest.java |   6 +-
 .../jdbc/test/junit/rules/MySqlConnectionRule.java |  23 +-
 .../test/junit/rules/PostgresConnectionRule.java   |  18 +-
 .../junit/rules/SqlDatabaseConnectionRule.java     |  61 +++--
 .../apache/geode/connectors/jdbc => }/mysql.yml    |   6 +-
 .../geode/connectors/jdbc/internal/mysql.yml       |  28 ---
 .../geode/connectors/jdbc/internal/postgres.yml    |  28 ---
 .../apache/geode/connectors/jdbc => }/postgres.yml |   4 +-
 33 files changed, 562 insertions(+), 474 deletions(-)

diff --git a/boms/geode-all-bom/src/test/resources/expected-pom.xml b/boms/geode-all-bom/src/test/resources/expected-pom.xml
index 7be7b37..83de33f 100644
--- a/boms/geode-all-bom/src/test/resources/expected-pom.xml
+++ b/boms/geode-all-bom/src/test/resources/expected-pom.xml
@@ -590,18 +590,6 @@
         <scope>compile</scope>
       </dependency>
       <dependency>
-        <groupId>com.palantir.docker.compose</groupId>
-        <artifactId>docker-compose-rule-core</artifactId>
-        <version>0.31.1</version>
-        <scope>compile</scope>
-      </dependency>
-      <dependency>
-        <groupId>com.palantir.docker.compose</groupId>
-        <artifactId>docker-compose-rule-junit4</artifactId>
-        <version>0.31.1</version>
-        <scope>compile</scope>
-      </dependency>
-      <dependency>
         <groupId>com.pholser</groupId>
         <artifactId>junit-quickcheck-core</artifactId>
         <version>0.9.1</version>
diff --git a/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy b/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
index f6a655a..1237159 100644
--- a/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
+++ b/buildSrc/src/main/groovy/org/apache/geode/gradle/plugins/DependencyConstraints.groovy
@@ -188,11 +188,6 @@ class DependencyConstraints implements Plugin<Project> {
       entry('json-path')
     }
 
-    dependencySet(group: 'com.palantir.docker.compose', version: '0.31.1') {
-      entry('docker-compose-rule-core')
-      entry('docker-compose-rule-junit4')
-    }
-
     dependencySet(group: 'com.pholser', version: '0.9.1') {
       entry('junit-quickcheck-core')
       entry('junit-quickcheck-generators')
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index 24760ec..1e06c44 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -139,10 +139,6 @@ repositories {
   maven {
     url 'https://repo.gradle.org/gradle/libs-releases'
   }
-  // docker-compose-rule is published on bintray
-  maven {
-    url 'https://dl.bintray.com/palantir/releases'
-  }
 }
 
 def webServersDir = "$buildDir/generated-resources/webservers"
@@ -271,8 +267,7 @@ dependencies {
   // don't have it be the same version as the outer gradle version.
   acceptanceTestImplementation('org.gradle:gradle-tooling-api:5.1.1')
 
-  acceptanceTestImplementation('com.palantir.docker.compose:docker-compose-rule-core')
-  acceptanceTestImplementation('com.palantir.docker.compose:docker-compose-rule-junit4')
+  acceptanceTestImplementation('org.testcontainers:testcontainers')
 
   uiTestImplementation(project(':geode-core'))
   uiTestImplementation(project(':geode-dunit')) {
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNICQAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNICQAcceptanceTest.java
index e1943d5..2879235 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNICQAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNICQAcceptanceTest.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.client.sni;
 
-import static com.palantir.docker.compose.execution.DockerComposeExecArgument.arguments;
-import static com.palantir.docker.compose.execution.DockerComposeExecOption.options;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_TYPE;
@@ -26,19 +24,16 @@ import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.IOException;
 import java.net.URL;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.palantir.docker.compose.DockerComposeRule;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
-import org.junit.rules.TestRule;
 
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.Region;
@@ -56,22 +51,18 @@ import org.apache.geode.cache.query.CqListener;
 import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.cache.query.RegionNotFoundException;
-import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+import org.apache.geode.rules.DockerComposeRule;
 
 public class ClientSNICQAcceptanceTest {
 
   private static final URL DOCKER_COMPOSE_PATH =
       ClientSNICQAcceptanceTest.class.getResource("docker-compose.yml");
 
-  // Docker compose does not work on windows in CI. Ignore this test on windows
-  // Using a RuleChain to make sure we ignore the test before the rule comes into play
   @ClassRule
-  public static TestRule ignoreOnWindowsRule = new IgnoreOnWindowsRule();
-
-  @ClassRule
-  public static NotOnWindowsDockerRule docker =
-      new NotOnWindowsDockerRule(() -> DockerComposeRule.builder()
-          .file(DOCKER_COMPOSE_PATH.getPath()).build());
+  public static DockerComposeRule docker = new DockerComposeRule.Builder()
+      .file(DOCKER_COMPOSE_PATH.getPath())
+      .service("haproxy", 15443)
+      .build();
 
   private CqQuery cqTracker;
 
@@ -79,15 +70,13 @@ public class ClientSNICQAcceptanceTest {
   AtomicInteger eventUpdateCounter = new AtomicInteger(0);
   private ClientCache cache;
   private Region<String, Integer> region;
+  private static String trustStorePath;
 
   class SNICQListener implements CqListener {
 
-
     @Override
     public void onEvent(CqEvent cqEvent) {
       Operation queryOperation = cqEvent.getQueryOperation();
-
-
       if (queryOperation.isUpdate()) {
         eventUpdateCounter.incrementAndGet();
       } else if (queryOperation.isCreate()) {
@@ -101,25 +90,24 @@ public class ClientSNICQAcceptanceTest {
     }
   }
 
-  private static String trustStorePath;
-
   @BeforeClass
-  public static void beforeClass() throws IOException, InterruptedException {
+  public static void beforeClass() {
     trustStorePath =
         createTempFileFromResource(ClientSNICQAcceptanceTest.class,
             "geode-config/truststore.jks")
                 .getAbsolutePath();
-    docker.get().exec(options("-T"), "geode",
-        arguments("gfsh", "run", "--file=/geode/scripts/geode-starter.gfsh"));
-
+    docker.loggingExecForService("geode",
+        "gfsh", "run", "--file=/geode/scripts/geode-starter.gfsh");
   }
 
   @AfterClass
-  public static void afterClass() throws Exception {
-    String output =
-        docker.get().exec(options("-T"), "geode",
-            arguments("cat", "server-dolores/server-dolores.log"));
-    System.out.println("Server log file--------------------------------\n" + output);
+  public static void afterClass() {
+    printlog("server-dolores");
+  }
+
+  private static void printlog(String name) {
+    String output = docker.execForService("geode", "cat", name + "/" + name + ".log");
+    System.out.println(name + " log file--------------------------------\n" + output);
   }
 
   @Before
@@ -133,10 +121,7 @@ public class ClientSNICQAcceptanceTest {
     gemFireProps.setProperty(SSL_TRUSTSTORE_PASSWORD, "geode");
     gemFireProps.setProperty(SSL_ENDPOINT_IDENTIFICATION_ENABLED, "true");
 
-    int proxyPort = docker.get().containers()
-        .container("haproxy")
-        .port(15443)
-        .getExternalPort();
+    int proxyPort = docker.getExternalPortForService("haproxy", 15443);
     cache = new ClientCacheFactory(gemFireProps)
         .addPoolLocator("locator-maeve", 10334)
         .setPoolSocketFactory(ProxySocketFactories.sni("localhost",
@@ -179,10 +164,10 @@ public class ClientSNICQAcceptanceTest {
     // the CQ has been closed. StatArchiveReader has a main() that we can use to get a printout
     // of stat values
     await().untilAsserted(() -> {
-      String stats = docker.get().exec(options("-T"), "geode",
-          arguments("java", "-cp", "/geode/lib/geode-dependencies.jar",
-              "org.apache.geode.internal.statistics.StatArchiveReader",
-              "stat", "server-dolores/statArchive.gfs", "CqServiceStats.numCqsClosed"));
+      String stats = docker.execForService("geode",
+          "java", "-cp", "/geode/lib/geode-dependencies.jar",
+          "org.apache.geode.internal.statistics.StatArchiveReader",
+          "stat", "server-dolores/statArchive.gfs", "CqServiceStats.numCqsClosed");
       // the stat should transition from zero to one at some point
       assertThat(stats).contains("0.0 1.0");
     });
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNIDropProxyAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNIDropProxyAcceptanceTest.java
index 309f381..095697a 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNIDropProxyAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/ClientSNIDropProxyAcceptanceTest.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.client.sni;
 
-import static com.palantir.docker.compose.execution.DockerComposeExecArgument.arguments;
-import static com.palantir.docker.compose.execution.DockerComposeExecOption.options;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_TYPE;
@@ -31,9 +29,6 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.Properties;
 
-import com.palantir.docker.compose.DockerComposeRule;
-import com.palantir.docker.compose.execution.DockerComposeRunArgument;
-import com.palantir.docker.compose.execution.DockerComposeRunOption;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -45,18 +40,18 @@ import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.client.NoAvailableLocatorsException;
 import org.apache.geode.cache.client.proxy.ProxySocketFactories;
+import org.apache.geode.rules.DockerComposeRule;
 
 public class ClientSNIDropProxyAcceptanceTest {
 
   private static final URL DOCKER_COMPOSE_PATH =
       ClientSNIDropProxyAcceptanceTest.class.getResource("docker-compose.yml");
 
-  // Docker compose does not work on windows in CI. Ignore this test on windows
-  // Using a RuleChain to make sure we ignore the test before the rule comes into play
   @ClassRule
-  public static NotOnWindowsDockerRule docker =
-      new NotOnWindowsDockerRule(() -> DockerComposeRule.builder()
-          .file(DOCKER_COMPOSE_PATH.getPath()).build());
+  public static DockerComposeRule docker = new DockerComposeRule.Builder()
+      .file(DOCKER_COMPOSE_PATH.getPath())
+      .service("haproxy", 15443)
+      .build();
 
   private ClientCache cache;
 
@@ -69,8 +64,7 @@ public class ClientSNIDropProxyAcceptanceTest {
         createTempFileFromResource(ClientSNIDropProxyAcceptanceTest.class,
             "geode-config/truststore.jks")
                 .getAbsolutePath();
-    docker.get().exec(options("-T"), "geode",
-        arguments("gfsh", "run", "--file=/geode/scripts/geode-starter.gfsh"));
+    docker.execForService("geode", "gfsh", "run", "--file=/geode/scripts/geode-starter.gfsh");
   }
 
   @After
@@ -79,22 +73,20 @@ public class ClientSNIDropProxyAcceptanceTest {
   }
 
   @Test
-  public void performSimpleOperationsDropSNIProxy()
-      throws IOException,
-      InterruptedException {
+  public void performSimpleOperationsDropSNIProxy() {
     final Region<String, Integer> region = getRegion();
 
     region.put("Roy Hobbs", 9);
     assertThat(region.get("Roy Hobbs")).isEqualTo(9);
 
-    stopProxy();
+    pauseProxy();
 
     assertThatThrownBy(() -> region.get("Roy Hobbs"))
         .isInstanceOf(NoAvailableLocatorsException.class)
         .hasMessageContaining("Unable to connect to any locators in the list");
 
 
-    restartProxy();
+    unpauseProxy();
 
     await().untilAsserted(() -> assertThat(region.get("Roy Hobbs")).isEqualTo(9));
 
@@ -109,53 +101,12 @@ public class ClientSNIDropProxyAcceptanceTest {
 
   }
 
-  private void stopProxy() throws IOException, InterruptedException {
-    docker.get().containers()
-        .container("haproxy")
-        .stop();
+  private void pauseProxy() {
+    docker.pauseService("haproxy");
   }
 
-  private void restartProxy() throws IOException, InterruptedException {
-    restartProxyOnPreviousPort();
-    // Leave this commented here in case you need it for troubleshooting
-    // restartProxyOnDockerComposePort();
-  }
-
-  /**
-   * Use this variant to (re)start the container on whatever port(s) is specified in
-   * docker-compose.yml. Usually that would look something like:
-   *
-   * ports:
-   * - "15443:15443"
-   *
-   * Leave this unused method here for troubleshooting.
-   */
-  private void restartProxyOnDockerComposePort() throws IOException, InterruptedException {
-    docker.get().containers()
-        .container("haproxy")
-        .start();
-  }
-
-  /**
-   * Use this variant to (re)start the container whatever host port it was bound to before
-   * it was stopped. Usually you'll want the ports spec in docker-compose.yml to look like
-   * this when using this method (allowing Docker to initially choose a random host port
-   * to bind to):
-   *
-   * ports:
-   * - "15443"
-   */
-  private void restartProxyOnPreviousPort() throws IOException, InterruptedException {
-    /*
-     * docker-compose run needs -d to daemonize the container (fork the process and return control
-     * to this process). The first time we ran the HAproxy container, we let it pick the host port
-     * to bind on. This time, we want it to bind to that same host port (proxyPort). The syntax
-     * for the --publish argument is <host-port>:<internal-port> in this case.
-     */
-    docker.get().run(
-        DockerComposeRunOption.options("-d", "--publish", String.format("%d:15443", proxyPort)),
-        "haproxy",
-        DockerComposeRunArgument.arguments("haproxy", "-f", "/usr/local/etc/haproxy/haproxy.cfg"));
+  private void unpauseProxy() {
+    docker.unpauseService("haproxy");
   }
 
   public Region<String, Integer> getRegion() {
@@ -168,10 +119,7 @@ public class ClientSNIDropProxyAcceptanceTest {
     gemFireProps.setProperty(SSL_TRUSTSTORE_PASSWORD, "geode");
     gemFireProps.setProperty(SSL_ENDPOINT_IDENTIFICATION_ENABLED, "true");
 
-    proxyPort = docker.get().containers()
-        .container("haproxy")
-        .port(15443)
-        .getExternalPort();
+    proxyPort = docker.getExternalPortForService("haproxy", 15443);
 
     ensureCacheClosed();
 
@@ -181,7 +129,7 @@ public class ClientSNIDropProxyAcceptanceTest {
             proxyPort))
         .setPoolSubscriptionEnabled(true)
         .create();
-    return (Region<String, Integer>) cache.<String, Integer>createClientRegionFactory(
+    return cache.<String, Integer>createClientRegionFactory(
         ClientRegionShortcut.PROXY)
         .create("jellyfish");
   }
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/DualServerSNIAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/DualServerSNIAcceptanceTest.java
index 7b08be2..6b14825 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/DualServerSNIAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/DualServerSNIAcceptanceTest.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.client.sni;
 
-import static com.palantir.docker.compose.execution.DockerComposeExecArgument.arguments;
-import static com.palantir.docker.compose.execution.DockerComposeExecOption.options;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_TYPE;
@@ -29,7 +27,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import java.net.URL;
 import java.util.Properties;
 
-import com.palantir.docker.compose.DockerComposeRule;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -41,10 +38,11 @@ import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.client.proxy.ProxySocketFactories;
+import org.apache.geode.rules.DockerComposeRule;
 
 /**
- * These tests run against a 2-server, 1-locator Geode cluster. The servers and locator run inside
- * a (single) Docker container and are not route-able from the host (where this JUnit test is
+ * These tests run against a 2-server, 1-locator Geode cluster. The servers and locator run inside a
+ * (single) Docker container and are not route-able from the host (where this JUnit test is
  * running). Another Docker container is running the HAProxy image and it's set up as an SNI
  * gateway. The test connects to the gateway via SNI and the gateway (in one Docker container)
  * forwards traffic to Geode members (running in the other Docker container).
@@ -53,29 +51,28 @@ import org.apache.geode.cache.client.proxy.ProxySocketFactories;
  * groups: group-dolores, and group-clementine, respectively. Also each server has a separate
  * REPLICATE region on it: region-dolores, and region-clementine, respectively.
  *
- * This test creates a connection pool to each group in turn. For that group, the test verifies
- * it can update data to the region of interest. There's also a pair of negative tests that verify
- * the correct exception is thrown when an attempt is made to operate on an unreachable region.
+ * This test creates a connection pool to each group in turn. For that group, the test verifies it
+ * can update data to the region of interest. There's also a pair of negative tests that verify the
+ * correct exception is thrown when an attempt is made to operate on an unreachable region.
  */
 public class DualServerSNIAcceptanceTest {
 
   private static final URL DOCKER_COMPOSE_PATH =
       SingleServerSNIAcceptanceTest.class.getResource("docker-compose.yml");
 
-  // Docker compose does not work on windows in CI. Ignore this test on windows
-  // Using a RuleChain to make sure we ignore the test before the rule comes into play
   @ClassRule
-  public static NotOnWindowsDockerRule docker =
-      new NotOnWindowsDockerRule(() -> DockerComposeRule.builder()
-          .file(DOCKER_COMPOSE_PATH.getPath()).build());
+  public static DockerComposeRule docker = new DockerComposeRule.Builder()
+      .file(DOCKER_COMPOSE_PATH.getPath())
+      .service("haproxy", 15443)
+      .build();
 
   private static Properties clientCacheProperties;
   private ClientCache cache;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    docker.get().exec(options("-T"), "geode",
-        arguments("gfsh", "run", "--file=/geode/scripts/geode-starter-2.gfsh"));
+    docker.loggingExecForService("geode",
+        "gfsh", "run", "--file=/geode/scripts/geode-starter-2.gfsh");
 
     final String trustStorePath =
         createTempFileFromResource(SingleServerSNIAcceptanceTest.class,
@@ -135,10 +132,7 @@ public class DualServerSNIAcceptanceTest {
    * modifies cache field as a side-effect
    */
   private Region<String, String> getRegion(final String groupName, final String regionName) {
-    final int proxyPort = docker.get().containers()
-        .container("haproxy")
-        .port(15443)
-        .getExternalPort();
+    final int proxyPort = docker.getExternalPortForService("haproxy", 15443);
     ensureCacheClosed();
     cache = new ClientCacheFactory(clientCacheProperties)
         .addPoolLocator("locator-maeve", 10334)
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/NotOnWindowsDockerRule.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/NotOnWindowsDockerRule.java
deleted file mode 100644
index 0e90fc1..0000000
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/NotOnWindowsDockerRule.java
+++ /dev/null
@@ -1,57 +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.geode.client.sni;
-
-import java.util.function.Supplier;
-
-import com.palantir.docker.compose.DockerComposeRule;
-import org.apache.commons.lang3.SystemUtils;
-import org.junit.Assume;
-import org.junit.rules.ExternalResource;
-
-/**
- * A rule that wraps {@link DockerComposeRule} in such a way that docker
- * tests will be ignored on the windows platform.
- *
- * Provide the code to build a DockerComposeRule in the constructor to this rule,
- * and access the rule later in your test with the {@link #get()} method.
- */
-public class NotOnWindowsDockerRule extends ExternalResource {
-
-  private final Supplier<DockerComposeRule> dockerRuleSupplier;
-  private DockerComposeRule docker;
-
-  public NotOnWindowsDockerRule(Supplier<DockerComposeRule> dockerRuleSupplier) {
-    this.dockerRuleSupplier = dockerRuleSupplier;
-  }
-
-  @Override
-  protected void before() throws Throwable {
-    Assume.assumeFalse(SystemUtils.IS_OS_WINDOWS);
-    this.docker = dockerRuleSupplier.get();
-    docker.before();
-  }
-
-  @Override
-  protected void after() {
-    if (docker != null) {
-      docker.after();
-    }
-  }
-
-  public DockerComposeRule get() {
-    return docker;
-  }
-}
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/SingleServerSNIAcceptanceTest.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/SingleServerSNIAcceptanceTest.java
index ccca1c7..1afbd59 100644
--- a/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/SingleServerSNIAcceptanceTest.java
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/client/sni/SingleServerSNIAcceptanceTest.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.client.sni;
 
-import static com.palantir.docker.compose.execution.DockerComposeExecArgument.arguments;
-import static com.palantir.docker.compose.execution.DockerComposeExecOption.options;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENABLED_COMPONENTS;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_ENDPOINT_IDENTIFICATION_ENABLED;
 import static org.apache.geode.distributed.ConfigurationProperties.SSL_KEYSTORE_TYPE;
@@ -25,14 +23,12 @@ import static org.apache.geode.distributed.ConfigurationProperties.SSL_TRUSTSTOR
 import static org.apache.geode.test.util.ResourceUtils.createTempFileFromResource;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.IOException;
 import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
-import com.palantir.docker.compose.DockerComposeRule;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -45,6 +41,7 @@ import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.client.proxy.ProxySocketFactories;
 import org.apache.geode.cache.query.SelectResults;
 import org.apache.geode.internal.cache.tier.sockets.BaseCommand;
+import org.apache.geode.rules.DockerComposeRule;
 
 /**
  * This test runs against a 1-server, 1-locator Geode cluster. The server and locator run inside
@@ -61,22 +58,20 @@ public class SingleServerSNIAcceptanceTest {
   private static final URL DOCKER_COMPOSE_PATH =
       SingleServerSNIAcceptanceTest.class.getResource("docker-compose.yml");
 
-  // Docker compose does not work on windows in CI. Ignore this test on windows
-  // Using a RuleChain to make sure we ignore the test before the rule comes into play
   @ClassRule
-  public static NotOnWindowsDockerRule docker =
-      new NotOnWindowsDockerRule(() -> DockerComposeRule.builder()
-          .file(DOCKER_COMPOSE_PATH.getPath()).build());
+  public static DockerComposeRule docker = new DockerComposeRule.Builder()
+      .file(DOCKER_COMPOSE_PATH.getPath())
+      .service("haproxy", 15443)
+      .build();
 
   private static ClientCache cache;
   private static Region<String, String> region;
   private static Map<String, String> bulkData;
 
   @BeforeClass
-  public static void beforeClass() throws IOException, InterruptedException {
+  public static void beforeClass() {
     // start up server/locator processes and initialize the server cache
-    docker.get().exec(options("-T"), "geode",
-        arguments("gfsh", "run", "--file=/geode/scripts/geode-starter.gfsh"));
+    docker.execForService("geode", "gfsh", "run", "--file=/geode/scripts/geode-starter.gfsh");
 
     final String trustStorePath =
         createTempFileFromResource(SingleServerSNIAcceptanceTest.class,
@@ -103,8 +98,7 @@ public class SingleServerSNIAcceptanceTest {
 
   @AfterClass
   public static void afterClass() throws Exception {
-    String logs = docker.get().exec(options("-T"), "geode",
-        arguments("cat", "server-dolores/server-dolores.log"));
+    String logs = docker.execForService("geode", "cat", "server-dolores/server-dolores.log");
     System.out.println("server logs------------------------------------------");
     System.out.println(logs);
 
@@ -190,10 +184,7 @@ public class SingleServerSNIAcceptanceTest {
   }
 
   protected static ClientCache getClientCache(Properties properties) {
-    int proxyPort = docker.get().containers()
-        .container("haproxy")
-        .port(15443)
-        .getExternalPort();
+    int proxyPort = docker.getExternalPortForService("haproxy", 15443);
     return new ClientCacheFactory(properties)
         .addPoolLocator("locator-maeve", 10334)
         .setPoolSocketFactory(ProxySocketFactories.sni("localhost",
diff --git a/geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java b/geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
new file mode 100644
index 0000000..912b1d6
--- /dev/null
+++ b/geode-assembly/src/acceptanceTest/java/org/apache/geode/rules/DockerComposeRule.java
@@ -0,0 +1,268 @@
+/*
+ * 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.geode.rules;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.github.dockerjava.api.DockerClient;
+import com.github.dockerjava.api.command.ExecCreateCmdResponse;
+import org.apache.logging.log4j.Logger;
+import org.junit.rules.ExternalResource;
+import org.junit.rules.RuleChain;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.DockerClientFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.ContainerState;
+import org.testcontainers.containers.DockerComposeContainer;
+import org.testcontainers.containers.output.BaseConsumer;
+import org.testcontainers.containers.output.FrameConsumerResultCallback;
+import org.testcontainers.containers.output.OutputFrame;
+import org.testcontainers.containers.output.ToStringConsumer;
+
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.test.junit.rules.IgnoreOnWindowsRule;
+
+/**
+ * This class assists in managing the lifecycle of a cluster, launched via a docker-compose
+ * configuration file, for testing. For example:
+ *
+ * <pre>
+ *
+ * &#64;ClassRule
+ * public static DockerComposeRule cluster = new DockerComposeRule().Builder()
+ *     .file("/home/bob/test/docker-compose.yml")
+ *     .service("haproxy", 15223)
+ *     .build();
+ *
+ * // Get the exposed port for haproxy
+ * cluster.getExternalPortForService("haproxy", 15223);
+ * </pre>
+ *
+ * Some limitations are as follows:
+ * <ul>
+ * <li>{@code testcontainers} does not support using {@code container_name:} attributes. If you
+ * need your container to be named explicitly, use {@link DockerComposeRule#setContainerName}</li>
+ * <li>Do not use the {@code expose} directives in your yaml file. Instead use
+ * {@link DockerComposeRule.Builder#service}
+ * to expose the relevant service and port.</li>
+ * <li>For now, this rule only handles a single instance of a service.</li>
+ * </ul>
+ *
+ * @see <a href=
+ *      "https://www.testcontainers.org/modules/docker_compose/">https://www.testcontainers.org/modules/docker_compose/</a>
+ */
+public class DockerComposeRule extends ExternalResource {
+
+  private static final Logger logger = LogService.getLogger();
+
+  private final RuleChain delegate;
+  private final String composeFile;
+  private final Map<String, List<Integer>> exposedServices;
+  private DockerComposeContainer<?> composeContainer;
+
+  public DockerComposeRule(String composeFile, Map<String, List<Integer>> exposedServices) {
+    this.composeFile = composeFile;
+    this.exposedServices = exposedServices;
+
+    // Docker compose does not work on windows in CI. Ignore this test on windows using a
+    // RuleChain to make sure we ignore the test before the rule comes into play.
+    delegate = RuleChain.outerRule(new IgnoreOnWindowsRule());
+  }
+
+  @Override
+  public Statement apply(Statement base, Description description) {
+    Statement containStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        composeContainer = new DockerComposeContainer<>("compose", new File(composeFile));
+        exposedServices.forEach((service, ports) -> ports
+            .forEach(p -> composeContainer.withExposedService(service, p)));
+
+        composeContainer.start();
+
+        try {
+          base.evaluate();
+        } finally {
+          composeContainer.stop();
+        }
+      }
+    };
+
+    return delegate.apply(containStatement, description);
+  }
+
+  /**
+   * When used with compose, testcontainers does not allow one to have a 'container_name'
+   * attribute in the compose file. This means that container names end up looking something like:
+   * {@code project_service_index}. When a container performs a reverse IP lookup it will get a
+   * hostname that looks something like {@code projectjkh_db_1.my-network}. This can be a problem
+   * since this hostname is not RFC compliant as it contains underscores. This may cause problems
+   * in particular with SSL.
+   *
+   * @param serviceName the service who's container name to change
+   * @param newName the new container name
+   *
+   * @throws IllegalArgumentException if the service cannot be found
+   */
+  public void setContainerName(String serviceName, String newName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+    dockerClient.renameContainerCmd(containerId).withName(newName).exec();
+  }
+
+  /**
+   * Execute a command in a service container
+   *
+   * @return the stdout of the container if the command was successful, else the stderr
+   */
+  public String execForService(String serviceName, String... command) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    Container.ExecResult result;
+    try {
+      result = container.execInContainer(command);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    return result.getExitCode() == 0 ? result.getStdout() : result.getStderr();
+  }
+
+  /**
+   * Execute a command in a service container, logging the output
+   *
+   * @return the exit code of the command
+   */
+  public Long loggingExecForService(String serviceName, String... command) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+
+    String containerId = container.getContainerId();
+    String containerName = container.getContainerInfo().getName();
+
+    logger.info("{}: Running 'exec' command: {}", containerName, command);
+
+    DockerClient dockerClient = DockerClientFactory.instance().client();
+
+    final ExecCreateCmdResponse execCreateCmdResponse = dockerClient.execCreateCmd(containerId)
+        .withAttachStdout(true).withAttachStderr(true).withCmd(command).exec();
+
+    final ToLogConsumer stdoutConsumer = new ToLogConsumer(serviceName, logger);
+    final ToLogConsumer stderrConsumer = new ToLogConsumer(serviceName, logger);
+
+    FrameConsumerResultCallback callback = new FrameConsumerResultCallback();
+    callback.addConsumer(OutputFrame.OutputType.STDOUT, stdoutConsumer);
+    callback.addConsumer(OutputFrame.OutputType.STDERR, stderrConsumer);
+
+    try {
+      dockerClient.execStartCmd(execCreateCmdResponse.getId()).exec(callback).awaitCompletion();
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+
+    return (long) dockerClient.inspectExecCmd(execCreateCmdResponse.getId()).exec().getExitCode();
+  }
+
+  /**
+   * Get the (ephemeral) exposed port for a service. This is the port that a test would use to
+   * connect.
+   *
+   * @param serviceName the service
+   * @param port the port (internal) which was exposed when building the rule
+   * @return the exposed port
+   */
+  public Integer getExternalPortForService(String serviceName, int port) {
+    return composeContainer.getServicePort(serviceName, port);
+  }
+
+  /**
+   * Pause a service. This is helpful to test failure conditions.
+   *
+   * @see <a href=
+   *      "https://docs.docker.com/engine/reference/commandline/pause/">https://docs.docker.com/engine/reference/commandline/pause/</a>
+   * @param serviceName the service to pause
+   */
+  public void pauseService(String serviceName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    DockerClientFactory.instance().client().pauseContainerCmd(container.getContainerId()).exec();
+  }
+
+  /**
+   * Unpause the service. This does not restart anything.
+   *
+   * @param serviceName the service to unpause
+   */
+  public void unpauseService(String serviceName) {
+    ContainerState container = composeContainer.getContainerByServiceName(serviceName + "_1")
+        .orElseThrow(() -> new IllegalArgumentException("Unknown service name: " + serviceName));
+    DockerClientFactory.instance().client().unpauseContainerCmd(container.getContainerId()).exec();
+  }
+
+  public static class Builder {
+    private String filePath;
+    private final Map<String, List<Integer>> exposedServices = new HashMap<>();
+
+    public Builder() {}
+
+    public DockerComposeRule build() {
+      return new DockerComposeRule(filePath, exposedServices);
+    }
+
+    public Builder file(String filePath) {
+      this.filePath = filePath;
+      return this;
+    }
+
+    public Builder service(String serviceName, Integer port) {
+      exposedServices.computeIfAbsent(serviceName, k -> new ArrayList<>()).add(port);
+      return this;
+    }
+  }
+
+  private static class ToLogConsumer extends BaseConsumer<ToStringConsumer> {
+    private boolean firstLine = true;
+    private final Logger logger;
+    private final String prefix;
+
+    public ToLogConsumer(String prefix, Logger logger) {
+      this.prefix = prefix;
+      this.logger = logger;
+    }
+
+    @Override
+    public void accept(OutputFrame outputFrame) {
+      if (outputFrame.getBytes() != null) {
+        if (!firstLine) {
+          logger.info("[{}]:", prefix);
+        }
+        logger.info("[{}]: {}", prefix, outputFrame.getUtf8String());
+        firstLine = false;
+      }
+    }
+  }
+
+}
diff --git a/geode-assembly/src/acceptanceTest/resources/org/apache/geode/client/sni/docker-compose.yml b/geode-assembly/src/acceptanceTest/resources/org/apache/geode/client/sni/docker-compose.yml
index dd52102..a037d07 100644
--- a/geode-assembly/src/acceptanceTest/resources/org/apache/geode/client/sni/docker-compose.yml
+++ b/geode-assembly/src/acceptanceTest/resources/org/apache/geode/client/sni/docker-compose.yml
@@ -17,12 +17,8 @@
 version: '3'
 services:
   geode:
-    container_name: 'geode'
     image: 'geode:develop'
     hostname: geode
-    expose:
-      - '10334'
-      - '40404'
     entrypoint: 'sh'
     command: '-c /geode/scripts/forever'
     networks:
@@ -31,10 +27,7 @@ services:
       - ./geode-config:/geode/config:ro
       - ./scripts:/geode/scripts
   haproxy:
-    container_name: 'haproxy'
     image: 'haproxy:2.1'
-    ports:
-      - "15443"
     networks:
       geode-sni-test:
     volumes:
diff --git a/geode-connectors/build.gradle b/geode-connectors/build.gradle
index 285c425..5b0c3dc 100644
--- a/geode-connectors/build.gradle
+++ b/geode-connectors/build.gradle
@@ -20,13 +20,6 @@ apply from: "${project.projectDir}/../gradle/warnings.gradle"
 
 evaluationDependsOn(":geode-core")
 
-repositories {
-  maven {
-    // docker-compose-rule is published on bintray
-    url 'https://dl.bintray.com/palantir/releases'
-  }
-}
-
 apply from: "${project.projectDir}/../gradle/publish-java.gradle"
 
 configurations {
@@ -118,9 +111,8 @@ dependencies {
   acceptanceTestImplementation('org.assertj:assertj-core')
   acceptanceTestImplementation('org.awaitility:awaitility')
   acceptanceTestImplementation('org.mockito:mockito-core')
-  acceptanceTestImplementation('com.palantir.docker.compose:docker-compose-rule-core')
   acceptanceTestImplementation('pl.pragmatists:JUnitParams')
-  acceptanceTestImplementation('com.palantir.docker.compose:docker-compose-rule-junit4')
+  acceptanceTestImplementation('org.testcontainers:testcontainers')
 
   acceptanceTestRuntimeOnly('mysql:mysql-connector-java')
   acceptanceTestRuntimeOnly('org.apache.derby:derby')
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcAsyncWriterIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcAsyncWriterIntegrationTest.java
index b1e05ee..d32fe5b 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcAsyncWriterIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcAsyncWriterIntegrationTest.java
@@ -25,6 +25,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.Supplier;
 
 import org.awaitility.core.ThrowingRunnable;
 import org.junit.After;
@@ -46,7 +47,6 @@ import org.apache.geode.pdx.WritablePdxInstance;
 
 public abstract class JdbcAsyncWriterIntegrationTest {
 
-  static final String DB_NAME = "test";
   private static final String REGION_TABLE_NAME = "employees";
 
   private InternalCache cache;
@@ -59,7 +59,7 @@ public abstract class JdbcAsyncWriterIntegrationTest {
   private Employee employee1;
   private Employee employee2;
   private final TestDataSourceFactory testDataSourceFactory =
-      new TestDataSourceFactory(getConnectionUrl());
+      new TestDataSourceFactory(getConnectionUrlSupplier());
 
   @Before
   public void setup() throws Exception {
@@ -107,7 +107,7 @@ public abstract class JdbcAsyncWriterIntegrationTest {
 
   public abstract Connection getConnection() throws SQLException;
 
-  public abstract String getConnectionUrl();
+  public abstract Supplier<String> getConnectionUrlSupplier();
 
   @Test
   public void validateJDBCAsyncWriterTotalEvents() throws RegionMappingExistsException {
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java
index 5d029d8..9e07bb1 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcDistributedTest.java
@@ -64,16 +64,15 @@ import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
  */
 public abstract class JdbcDistributedTest implements Serializable {
 
-  static final String DB_NAME = "test";
   private static final String TABLE_NAME = "employees";
   private static final String REGION_NAME = "employees";
   private static final String DATA_SOURCE_NAME = "TestDataSource";
 
   @Rule
-  public transient GfshCommandRule gfsh = new GfshCommandRule();
+  public GfshCommandRule gfsh = new GfshCommandRule();
 
   @Rule
-  public transient ClusterStartupRule startupRule = new ClusterStartupRule();
+  public ClusterStartupRule startupRule = new ClusterStartupRule();
 
   @Rule
   public SerializableTestName testName = new SerializableTestName();
@@ -84,13 +83,11 @@ public abstract class JdbcDistributedTest implements Serializable {
 
   private MemberVM dataserver;
   private MemberVM locator;
-  private String connectionUrl;
 
   @Before
   public void setup() throws Exception {
     locator = startupRule.startLocatorVM(0);
     gfsh.connectAndVerify(locator);
-    connectionUrl = getConnectionUrl();
   }
 
   public abstract Connection getConnection() throws SQLException;
@@ -98,7 +95,8 @@ public abstract class JdbcDistributedTest implements Serializable {
   public abstract String getConnectionUrl() throws IOException, InterruptedException;
 
   private void createTable() throws SQLException {
-    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
+    int locatorPort = locator.getPort();
+    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locatorPort));
     Connection connection = getConnection();
     Statement statement = connection.createStatement();
     statement.execute("Create Table " + TABLE_NAME
@@ -106,7 +104,8 @@ public abstract class JdbcDistributedTest implements Serializable {
   }
 
   private MemberVM createTableForGroup(int idx, String groupName) throws SQLException {
-    MemberVM server = startupRule.startServerVM(idx, groupName, locator.getPort());
+    int locatorPort = locator.getPort();
+    MemberVM server = startupRule.startServerVM(idx, groupName, locatorPort);
     Connection connection = getConnection();
     Statement statement = connection.createStatement();
     statement.execute("Create Table " + TABLE_NAME
@@ -114,8 +113,9 @@ public abstract class JdbcDistributedTest implements Serializable {
     return server;
   }
 
-  private MemberVM addServerForGroup(int idx, String groupName) throws SQLException {
-    MemberVM server = startupRule.startServerVM(idx, groupName, locator.getPort());
+  private MemberVM addServerForGroup(int idx, String groupName) {
+    int locatorPort = locator.getPort();
+    MemberVM server = startupRule.startServerVM(idx, groupName, locatorPort);
     return server;
   }
 
@@ -127,8 +127,9 @@ public abstract class JdbcDistributedTest implements Serializable {
   }
 
   private void createTableForAllSupportedFields() throws SQLException {
+    int locatorPort = locator.getPort();
     dataserver = startupRule.startServerVM(1,
-        x -> x.withConnectionToLocator(locator.getPort()).withPDXReadSerialized());
+        x -> x.withConnectionToLocator(locatorPort).withPDXReadSerialized());
     Connection connection = getConnection();
     DatabaseMetaData metaData = connection.getMetaData();
     String quote = metaData.getIdentifierQuoteString();
@@ -139,24 +140,11 @@ public abstract class JdbcDistributedTest implements Serializable {
   protected abstract void createSupportedFieldsTable(Statement statement, String tableName,
       String quote) throws SQLException;
 
-  private void insertNullDataForAllSupportedFieldsTable(String key) throws SQLException {
-    Connection connection = DriverManager.getConnection(connectionUrl);
-    DatabaseMetaData metaData = connection.getMetaData();
-    String quote = metaData.getIdentifierQuoteString();
-
-    String insertQuery =
-        "Insert into " + quote + TABLE_NAME + quote + " values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
-    System.out.println("### Query is :" + insertQuery);
-    PreparedStatement statement = connection.prepareStatement(insertQuery);
-    createNullStatement(key, statement);
-
-    statement.execute();
-  }
+  protected abstract void insertNullDataForAllSupportedFieldsTable(MemberVM dataserver, String key,
+      String tableName) throws SQLException;
 
-  protected abstract void createNullStatement(String key, PreparedStatement statement)
-      throws SQLException;
-
-  private void insertDataForAllSupportedFieldsTable(String key, ClassWithSupportedPdxFields data)
+  private static void insertDataForAllSupportedFieldsTable(String connectionUrl, String key,
+      ClassWithSupportedPdxFields data)
       throws SQLException {
     Connection connection = DriverManager.getConnection(connectionUrl);
     DatabaseMetaData metaData = connection.getMetaData();
@@ -189,7 +177,7 @@ public abstract class JdbcDistributedTest implements Serializable {
   }
 
   private void closeDB() throws SQLException {
-    try (Connection connection = DriverManager.getConnection(connectionUrl)) {
+    try (Connection connection = getConnection()) {
       DatabaseMetaData metaData = connection.getMetaData();
       String quote = metaData.getIdentifierQuoteString();
       try (Statement statement = connection.createStatement()) {
@@ -320,8 +308,9 @@ public abstract class JdbcDistributedTest implements Serializable {
     createJdbcDataSource();
     createMapping(REGION_NAME, DATA_SOURCE_NAME, true);
     alterTable();
+    int locatorPort = locator.getPort();
     assertThatThrownBy(
-        () -> startupRule.startServerVM(3, x -> x.withConnectionToLocator(locator.getPort())))
+        () -> startupRule.startServerVM(3, x -> x.withConnectionToLocator(locatorPort)))
             .hasCauseExactlyInstanceOf(JdbcConnectorException.class).hasStackTraceContaining(
                 "Jdbc mapping for \"employees\" does not match table definition, check logs for more details.");
   }
@@ -429,8 +418,9 @@ public abstract class JdbcDistributedTest implements Serializable {
     createPartitionedRegionUsingGfsh();
     createJdbcDataSource();
     createMapping(REGION_NAME, DATA_SOURCE_NAME, false);
+    int locatorPort = locator.getPort();
     MemberVM server3 =
-        startupRule.startServerVM(3, x -> x.withConnectionToLocator(locator.getPort()));
+        startupRule.startServerVM(3, x -> x.withConnectionToLocator(locatorPort));
     server3.invoke(() -> {
       RegionMapping mapping =
           ClusterStartupRule.getCache().getService(JdbcConnectorService.class)
@@ -457,7 +447,9 @@ public abstract class JdbcDistributedTest implements Serializable {
 
   @Test
   public void verifyDateToDate() throws Exception {
-    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
+    int locatorPort = locator.getPort();
+    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locatorPort));
+    String connectionUrl = getConnectionUrl();
     dataserver.invoke(() -> {
       Connection connection = DriverManager.getConnection(connectionUrl);
       Statement statement = connection.createStatement();
@@ -492,7 +484,9 @@ public abstract class JdbcDistributedTest implements Serializable {
 
   @Test
   public void verifyDateToTime() throws Exception {
-    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
+    int locatorPort = locator.getPort();
+    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locatorPort));
+    String connectionUrl = getConnectionUrl();
     dataserver.invoke(() -> {
       Connection connection = DriverManager.getConnection(connectionUrl);
       Statement statement = connection.createStatement();
@@ -527,8 +521,9 @@ public abstract class JdbcDistributedTest implements Serializable {
 
   @Test
   public void verifyDateToTimestamp() throws Exception {
-    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locator.getPort()));
-    createTableWithTimeStamp(dataserver, connectionUrl, TABLE_NAME, TestDate.DATE_FIELD_NAME);
+    int locatorPort = locator.getPort();
+    dataserver = startupRule.startServerVM(1, x -> x.withConnectionToLocator(locatorPort));
+    createTableWithTimeStamp(dataserver, getConnectionUrl(), TABLE_NAME, TestDate.DATE_FIELD_NAME);
 
     createReplicatedRegionUsingGfsh();
     createJdbcDataSource();
@@ -571,6 +566,7 @@ public abstract class JdbcDistributedTest implements Serializable {
     createReplicatedRegionUsingGfsh();
     createJdbcDataSource();
     createMapping(REGION_NAME, DATA_SOURCE_NAME, true);
+    String finalUrl = getConnectionUrl();
     dataserver.invoke(() -> {
       PdxInstance pdxEmployee1 =
           ClusterStartupRule.getCache().createPdxInstanceFactory(Employee.class.getName())
@@ -578,7 +574,7 @@ public abstract class JdbcDistributedTest implements Serializable {
 
       String key = "emp1";
       ClusterStartupRule.getCache().getRegion(REGION_NAME).put(key, pdxEmployee1);
-      assertTableHasEmployeeData(1, pdxEmployee1, key);
+      assertTableHasEmployeeData(finalUrl, 1, pdxEmployee1, key);
     });
   }
 
@@ -588,6 +584,7 @@ public abstract class JdbcDistributedTest implements Serializable {
     createReplicatedRegionUsingGfsh();
     createJdbcDataSource();
     createMapping(REGION_NAME, DATA_SOURCE_NAME, false);
+    String finalUrl = getConnectionUrl();
     dataserver.invoke(() -> {
       PdxInstance pdxEmployee1 =
           ClusterStartupRule.getCache().createPdxInstanceFactory(Employee.class.getName())
@@ -596,7 +593,7 @@ public abstract class JdbcDistributedTest implements Serializable {
       String key = "emp1";
       ClusterStartupRule.getCache().getRegion(REGION_NAME).put(key, pdxEmployee1);
       await().untilAsserted(() -> {
-        assertTableHasEmployeeData(1, pdxEmployee1, key);
+        assertTableHasEmployeeData(finalUrl, 1, pdxEmployee1, key);
       });
     });
   }
@@ -607,6 +604,7 @@ public abstract class JdbcDistributedTest implements Serializable {
     createPartitionedRegionUsingGfsh();
     createJdbcDataSource();
     createMapping(REGION_NAME, DATA_SOURCE_NAME, false);
+    String finalUrl = getConnectionUrl();
     dataserver.invoke(() -> {
       PdxInstance pdxEmployee1 =
           ClusterStartupRule.getCache().createPdxInstanceFactory(Employee.class.getName())
@@ -615,7 +613,7 @@ public abstract class JdbcDistributedTest implements Serializable {
       String key = "emp1";
       ClusterStartupRule.getCache().getRegion(REGION_NAME).put(key, pdxEmployee1);
       await().untilAsserted(() -> {
-        assertTableHasEmployeeData(1, pdxEmployee1, key);
+        assertTableHasEmployeeData(finalUrl, 1, pdxEmployee1, key);
       });
     });
   }
@@ -803,8 +801,9 @@ public abstract class JdbcDistributedTest implements Serializable {
         new ClassWithSupportedPdxFields(key, true, (byte) 1, (short) 2,
             3, 4, 5.5f, 6.0, "BigEmp", new Date(0), "BigEmpObject", new byte[] {1, 2}, 'c');
 
+    String finalUrl = getConnectionUrl();
     dataserver.invoke(() -> {
-      insertDataForAllSupportedFieldsTable(key, value);
+      insertDataForAllSupportedFieldsTable(finalUrl, key, value);
     });
 
     client.invoke(() -> {
@@ -830,9 +829,7 @@ public abstract class JdbcDistributedTest implements Serializable {
     String key = "id1";
     ClassWithSupportedPdxFields value = new ClassWithSupportedPdxFields(key);
 
-    dataserver.invoke(() -> {
-      insertNullDataForAllSupportedFieldsTable(key);
-    });
+    insertNullDataForAllSupportedFieldsTable(dataserver, key, TABLE_NAME);
 
     client.invoke(() -> {
       ClusterStartupRule.getClientCache().registerPdxMetaData(new ClassWithSupportedPdxFields());
@@ -846,9 +843,10 @@ public abstract class JdbcDistributedTest implements Serializable {
   }
 
   private ClientVM getClientVM() throws Exception {
+    int locatorPort = locator.getPort();
     SerializableConsumerIF<ClientCacheFactory> cacheSetup = cf -> {
       System.setProperty(AutoSerializableManager.NO_HARDCODED_EXCLUDES_PARAM, "true");
-      cf.addPoolLocator("localhost", locator.getPort());
+      cf.addPoolLocator("localhost", locatorPort);
       cf.setPdxSerializer(
           new ReflectionBasedAutoSerializer(ClassWithSupportedPdxFields.class.getName()));
     };
@@ -862,9 +860,9 @@ public abstract class JdbcDistributedTest implements Serializable {
     });
   }
 
-  private void createJdbcDataSource() {
+  private void createJdbcDataSource() throws Exception {
     final String commandStr =
-        "create data-source --pooled --name=" + DATA_SOURCE_NAME + " --url=" + connectionUrl;
+        "create data-source --pooled --name=" + DATA_SOURCE_NAME + " --url=" + getConnectionUrl();
     gfsh.executeAndAssertThat(commandStr).statusIsSuccess();
   }
 
@@ -950,7 +948,8 @@ public abstract class JdbcDistributedTest implements Serializable {
     }
   }
 
-  private void assertTableHasEmployeeData(int size, PdxInstance employee, String key)
+  private static void assertTableHasEmployeeData(String connectionUrl, int size,
+      PdxInstance employee, String key)
       throws SQLException {
     Connection connection = DriverManager.getConnection(connectionUrl);
     Statement statement = connection.createStatement();
@@ -966,7 +965,7 @@ public abstract class JdbcDistributedTest implements Serializable {
     assertThat(resultSet.getObject("age")).isEqualTo(employee.getField("age"));
   }
 
-  private int getRowCount(Statement stmt, String tableName) {
+  private static int getRowCount(Statement stmt, String tableName) {
     try {
       ResultSet resultSet = stmt.executeQuery("select count(*) from " + tableName);
       resultSet.next();
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcLoaderIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcLoaderIntegrationTest.java
index 1a5ab83..18eb6ad 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcLoaderIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcLoaderIntegrationTest.java
@@ -26,6 +26,7 @@ import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.List;
+import java.util.function.Supplier;
 
 import org.junit.After;
 import org.junit.Before;
@@ -50,7 +51,6 @@ import org.apache.geode.pdx.internal.AutoSerializableManager;
 
 public abstract class JdbcLoaderIntegrationTest {
 
-  static final String DB_NAME = "test";
   protected static final String SCHEMA_NAME = "mySchema";
   protected static final String REGION_TABLE_NAME = "employees";
 
@@ -62,7 +62,7 @@ public abstract class JdbcLoaderIntegrationTest {
   public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
 
   private final TestDataSourceFactory testDataSourceFactory =
-      new TestDataSourceFactory(getConnectionUrl());
+      new TestDataSourceFactory(getConnectionUrlSupplier());
 
   @Before
   public void setUp() throws Exception {
@@ -84,7 +84,7 @@ public abstract class JdbcLoaderIntegrationTest {
 
   public abstract Connection getConnection() throws SQLException;
 
-  public abstract String getConnectionUrl();
+  public abstract Supplier<String> getConnectionUrlSupplier();
 
   protected abstract void createClassWithSupportedPdxFieldsTable(Statement statement,
       String tableName) throws SQLException;
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcWriterIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcWriterIntegrationTest.java
index 905f415..8140dd0 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcWriterIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/JdbcWriterIntegrationTest.java
@@ -26,6 +26,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Supplier;
 
 import javax.sql.DataSource;
 
@@ -49,7 +50,6 @@ import org.apache.geode.pdx.WritablePdxInstance;
 
 public abstract class JdbcWriterIntegrationTest {
 
-  static final String DB_NAME = "test";
   protected static final String SCHEMA_NAME = "mySchema";
   protected static final String REGION_TABLE_NAME = "employees";
 
@@ -63,7 +63,7 @@ public abstract class JdbcWriterIntegrationTest {
   protected Employee employee1;
   protected Employee employee2;
   protected final TestDataSourceFactory testDataSourceFactory =
-      new TestDataSourceFactory(getConnectionUrl());
+      new TestDataSourceFactory(getConnectionUrlSupplier());
   protected String catalog;
   protected String schema;
 
@@ -138,7 +138,7 @@ public abstract class JdbcWriterIntegrationTest {
 
   public abstract Connection getConnection() throws SQLException;
 
-  public abstract String getConnectionUrl();
+  public abstract Supplier<String> getConnectionUrlSupplier();
 
   private void closeDB() throws Exception {
     if (statement == null) {
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcAsyncWriterIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcAsyncWriterIntegrationTest.java
index 5ae2760..3ecf21a 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcAsyncWriterIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcAsyncWriterIntegrationTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.connectors.jdbc;
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.function.Supplier;
 
 import org.junit.ClassRule;
 
@@ -26,11 +27,11 @@ import org.apache.geode.connectors.jdbc.test.junit.rules.MySqlConnectionRule;
 public class MySqlJdbcAsyncWriterIntegrationTest extends JdbcAsyncWriterIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      MySqlJdbcAsyncWriterIntegrationTest.class.getResource("mysql.yml");
+      MySqlJdbcAsyncWriterIntegrationTest.class.getResource("/mysql.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new MySqlConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(3306).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new MySqlConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -38,7 +39,7 @@ public class MySqlJdbcAsyncWriterIntegrationTest extends JdbcAsyncWriterIntegrat
   }
 
   @Override
-  public String getConnectionUrl() {
-    return dbRule.getConnectionUrl();
+  public Supplier<String> getConnectionUrlSupplier() {
+    return () -> dbRule.getConnectionUrl();
   }
 }
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java
index 72fba76..8933044 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcDistributedTest.java
@@ -16,6 +16,7 @@ package org.apache.geode.connectors.jdbc;
 
 import java.net.URL;
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
@@ -24,8 +25,8 @@ import java.sql.Types;
 
 import org.junit.ClassRule;
 
+import org.apache.geode.connectors.jdbc.test.junit.rules.DatabaseConnectionRule;
 import org.apache.geode.connectors.jdbc.test.junit.rules.MySqlConnectionRule;
-import org.apache.geode.connectors.jdbc.test.junit.rules.SqlDatabaseConnectionRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 
 /**
@@ -34,19 +35,11 @@ import org.apache.geode.test.dunit.rules.MemberVM;
 public class MySqlJdbcDistributedTest extends JdbcDistributedTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      MySqlJdbcDistributedTest.class.getResource("mysql.yml");
+      MySqlJdbcDistributedTest.class.getResource("/mysql.yml");
 
   @ClassRule
-  public static transient SqlDatabaseConnectionRule dbRule = createConnectionRule();
-
-  private static SqlDatabaseConnectionRule createConnectionRule() {
-    try {
-      return new MySqlConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath())
-          .serviceName("db").port(3306).database(DB_NAME).build();
-    } catch (IllegalStateException e) {
-      return null;
-    }
-  }
+  public static DatabaseConnectionRule dbRule =
+      new MySqlConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -55,7 +48,7 @@ public class MySqlJdbcDistributedTest extends JdbcDistributedTest {
 
   @Override
   public String getConnectionUrl() {
-    return dbRule == null ? null : dbRule.getConnectionUrl();
+    return dbRule.getConnectionUrl();
   }
 
   @Override
@@ -70,7 +63,26 @@ public class MySqlJdbcDistributedTest extends JdbcDistributedTest {
   }
 
   @Override
-  protected void createNullStatement(String key, PreparedStatement statement) throws SQLException {
+  protected void insertNullDataForAllSupportedFieldsTable(MemberVM dataserver, String key,
+      String tableName) {
+    String finalUrl = getConnectionUrl();
+    dataserver.invoke(() -> {
+      Connection connection = DriverManager.getConnection(finalUrl);
+      DatabaseMetaData metaData = connection.getMetaData();
+      String quote = metaData.getIdentifierQuoteString();
+
+      String insertQuery =
+          "Insert into " + quote + tableName + quote + " values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
+      System.out.println("### Query is :" + insertQuery);
+      PreparedStatement statement = connection.prepareStatement(insertQuery);
+      createNullStatement(key, statement);
+
+      statement.execute();
+    });
+  }
+
+  private static void createNullStatement(String key, PreparedStatement statement)
+      throws SQLException {
     statement.setObject(1, key);
     statement.setNull(2, Types.BOOLEAN);
     statement.setNull(3, Types.SMALLINT);
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcLoaderIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcLoaderIntegrationTest.java
index 19dc16e..f4746b7 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcLoaderIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcLoaderIntegrationTest.java
@@ -21,6 +21,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.Supplier;
 
 import org.junit.ClassRule;
 
@@ -32,11 +33,11 @@ import org.apache.geode.pdx.FieldType;
 public class MySqlJdbcLoaderIntegrationTest extends JdbcLoaderIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      MySqlJdbcLoaderIntegrationTest.class.getResource("mysql.yml");
+      MySqlJdbcLoaderIntegrationTest.class.getResource("/mysql.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new MySqlConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(3306).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new MySqlConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -44,8 +45,8 @@ public class MySqlJdbcLoaderIntegrationTest extends JdbcLoaderIntegrationTest {
   }
 
   @Override
-  public String getConnectionUrl() {
-    return dbRule.getConnectionUrl();
+  public Supplier<String> getConnectionUrlSupplier() {
+    return () -> dbRule.getConnectionUrl();
   }
 
   @Override
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcWriterIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcWriterIntegrationTest.java
index d38b230..2f555c5 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcWriterIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/MySqlJdbcWriterIntegrationTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.connectors.jdbc;
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.function.Supplier;
 
 import org.junit.ClassRule;
 
@@ -26,11 +27,11 @@ import org.apache.geode.connectors.jdbc.test.junit.rules.MySqlConnectionRule;
 public class MySqlJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      MySqlJdbcWriterIntegrationTest.class.getResource("mysql.yml");
+      MySqlJdbcWriterIntegrationTest.class.getResource("/mysql.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new MySqlConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(3306).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new MySqlConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -38,8 +39,8 @@ public class MySqlJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest {
   }
 
   @Override
-  public String getConnectionUrl() {
-    return dbRule.getConnectionUrl();
+  public Supplier<String> getConnectionUrlSupplier() {
+    return () -> dbRule.getConnectionUrl();
   }
 
   @Override
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcAsyncWriterIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcAsyncWriterIntegrationTest.java
index 99bb079..9c88043 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcAsyncWriterIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcAsyncWriterIntegrationTest.java
@@ -17,6 +17,7 @@ package org.apache.geode.connectors.jdbc;
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.function.Supplier;
 
 import org.junit.ClassRule;
 
@@ -26,11 +27,11 @@ import org.apache.geode.connectors.jdbc.test.junit.rules.PostgresConnectionRule;
 public class PostgresJdbcAsyncWriterIntegrationTest extends JdbcAsyncWriterIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      PostgresJdbcAsyncWriterIntegrationTest.class.getResource("postgres.yml");
+      PostgresJdbcAsyncWriterIntegrationTest.class.getResource("/postgres.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new PostgresConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(5432).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new PostgresConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -38,7 +39,7 @@ public class PostgresJdbcAsyncWriterIntegrationTest extends JdbcAsyncWriterInteg
   }
 
   @Override
-  public String getConnectionUrl() {
-    return dbRule.getConnectionUrl();
+  public Supplier<String> getConnectionUrlSupplier() {
+    return () -> dbRule.getConnectionUrl();
   }
 }
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java
index 1811c63..4cd0eff 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcDistributedTest.java
@@ -16,6 +16,8 @@ package org.apache.geode.connectors.jdbc;
 
 import java.net.URL;
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -23,8 +25,9 @@ import java.sql.Types;
 
 import org.junit.ClassRule;
 
+import org.apache.geode.connectors.jdbc.test.junit.rules.DatabaseConnectionRule;
 import org.apache.geode.connectors.jdbc.test.junit.rules.PostgresConnectionRule;
-import org.apache.geode.connectors.jdbc.test.junit.rules.SqlDatabaseConnectionRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
 
 /**
  * End-to-end dunits for jdbc connector
@@ -32,19 +35,11 @@ import org.apache.geode.connectors.jdbc.test.junit.rules.SqlDatabaseConnectionRu
 public class PostgresJdbcDistributedTest extends JdbcDistributedTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      PostgresJdbcDistributedTest.class.getResource("postgres.yml");
+      PostgresJdbcDistributedTest.class.getResource("/postgres.yml");
 
   @ClassRule
-  public static transient SqlDatabaseConnectionRule dbRule = createConnectionRule();
-
-  private static SqlDatabaseConnectionRule createConnectionRule() {
-    try {
-      return new PostgresConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath())
-          .serviceName("db").port(5432).database(DB_NAME).build();
-    } catch (IllegalStateException e) {
-      return null;
-    }
-  }
+  public static DatabaseConnectionRule dbRule =
+      new PostgresConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -68,7 +63,26 @@ public class PostgresJdbcDistributedTest extends JdbcDistributedTest {
   }
 
   @Override
-  protected void createNullStatement(String key, PreparedStatement statement) throws SQLException {
+  protected void insertNullDataForAllSupportedFieldsTable(MemberVM dataserver, String key,
+      String tableName) {
+    String finalUrl = getConnectionUrl();
+    dataserver.invoke(() -> {
+      Connection connection = DriverManager.getConnection(finalUrl);
+      DatabaseMetaData metaData = connection.getMetaData();
+      String quote = metaData.getIdentifierQuoteString();
+
+      String insertQuery =
+          "Insert into " + quote + tableName + quote + " values (" + "?,?,?,?,?,?,?,?,?,?,?,?,?)";
+      System.out.println("### Query is :" + insertQuery);
+      PreparedStatement statement = connection.prepareStatement(insertQuery);
+      createNullStatement(key, statement);
+
+      statement.execute();
+    });
+  }
+
+  private static void createNullStatement(String key, PreparedStatement statement)
+      throws SQLException {
     statement.setObject(1, key);
     statement.setNull(2, Types.BOOLEAN);
     statement.setNull(3, Types.SMALLINT);
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcLoaderIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcLoaderIntegrationTest.java
index ed53c53..7586b72 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcLoaderIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcLoaderIntegrationTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.connectors.jdbc;
 
+import static org.apache.geode.connectors.jdbc.test.junit.rules.SqlDatabaseConnectionRule.DEFAULT_DB_NAME;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.net.URL;
@@ -23,6 +24,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.Supplier;
 
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -37,11 +39,11 @@ import org.apache.geode.pdx.PdxInstance;
 public class PostgresJdbcLoaderIntegrationTest extends JdbcLoaderIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      PostgresJdbcLoaderIntegrationTest.class.getResource("postgres.yml");
+      PostgresJdbcLoaderIntegrationTest.class.getResource("/postgres.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new PostgresConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(5432).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new PostgresConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -49,8 +51,8 @@ public class PostgresJdbcLoaderIntegrationTest extends JdbcLoaderIntegrationTest
   }
 
   @Override
-  public String getConnectionUrl() {
-    return dbRule.getConnectionUrl();
+  public Supplier<String> getConnectionUrlSupplier() {
+    return () -> dbRule.getConnectionUrl();
   }
 
   @Override
@@ -94,19 +96,21 @@ public class PostgresJdbcLoaderIntegrationTest extends JdbcLoaderIntegrationTest
 
   private void createEmployeeTableWithSchemaAndCatalog() throws Exception {
     statement.execute("CREATE SCHEMA " + SCHEMA_NAME);
-    statement.execute("Create Table " + DB_NAME + '.' + SCHEMA_NAME + '.' + REGION_TABLE_NAME
-        + " (id varchar(10) primary key not null, name varchar(10), age int)");
+    statement
+        .execute("Create Table " + DEFAULT_DB_NAME + '.' + SCHEMA_NAME + '.' + REGION_TABLE_NAME
+            + " (id varchar(10) primary key not null, name varchar(10), age int)");
   }
 
   @Test
   public void verifyGetWithSchemaCatalogAndPdxClassNameAndCompositeKey() throws Exception {
     createEmployeeTableWithSchemaAndCatalog();
     statement
-        .execute("Insert into " + DB_NAME + '.' + SCHEMA_NAME + '.' + REGION_TABLE_NAME
+        .execute("Insert into " + DEFAULT_DB_NAME + '.' + SCHEMA_NAME + '.' + REGION_TABLE_NAME
             + "(id, name, age) values('1', 'Emp1', 21)");
     String ids = "id,name";
     Region<String, Employee> region =
-        createRegionWithJDBCLoader(REGION_TABLE_NAME, Employee.class.getName(), ids, DB_NAME,
+        createRegionWithJDBCLoader(REGION_TABLE_NAME, Employee.class.getName(), ids,
+            DEFAULT_DB_NAME,
             SCHEMA_NAME, getEmployeeTableFieldMappings());
     createPdxType();
 
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcWriterIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcWriterIntegrationTest.java
index 4ba7209..fee9fd8 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcWriterIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/PostgresJdbcWriterIntegrationTest.java
@@ -14,12 +14,14 @@
  */
 package org.apache.geode.connectors.jdbc;
 
+import static org.apache.geode.connectors.jdbc.test.junit.rules.SqlDatabaseConnectionRule.DEFAULT_DB_NAME;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.function.Supplier;
 
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -30,11 +32,11 @@ import org.apache.geode.connectors.jdbc.test.junit.rules.PostgresConnectionRule;
 public class PostgresJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      PostgresJdbcWriterIntegrationTest.class.getResource("postgres.yml");
+      PostgresJdbcWriterIntegrationTest.class.getResource("/postgres.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new PostgresConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(5432).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new PostgresConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
@@ -42,8 +44,8 @@ public class PostgresJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest
   }
 
   @Override
-  public String getConnectionUrl() {
-    return dbRule.getConnectionUrl();
+  public Supplier<String> getConnectionUrlSupplier() {
+    return () -> dbRule.getConnectionUrl();
   }
 
   @Override
@@ -54,21 +56,22 @@ public class PostgresJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest
 
   protected void createTableWithCatalogAndSchema() throws SQLException {
     statement.execute("Create Schema " + SCHEMA_NAME);
-    statement.execute("Create Table " + DB_NAME + '.' + SCHEMA_NAME + '.' + REGION_TABLE_NAME
-        + " (id varchar(10) primary key not null, name varchar(10), age int)");
+    statement
+        .execute("Create Table " + DEFAULT_DB_NAME + '.' + SCHEMA_NAME + '.' + REGION_TABLE_NAME
+            + " (id varchar(10) primary key not null, name varchar(10), age int)");
   }
 
   @Test
   public void canDestroyFromTableWithCatalogAndSchema() throws Exception {
     createTableWithCatalogAndSchema();
-    sharedRegionSetup("id", DB_NAME, SCHEMA_NAME);
+    sharedRegionSetup("id", DEFAULT_DB_NAME, SCHEMA_NAME);
     employees.put("1", pdx1);
     employees.put("2", pdx2);
 
     employees.destroy("1");
 
     ResultSet resultSet =
-        statement.executeQuery("select * from " + DB_NAME + '.' + SCHEMA_NAME + '.'
+        statement.executeQuery("select * from " + DEFAULT_DB_NAME + '.' + SCHEMA_NAME + '.'
             + REGION_TABLE_NAME + " order by id asc");
     assertRecordMatchesEmployee(resultSet, "2", employee2);
     assertThat(resultSet.next()).isFalse();
@@ -77,12 +80,12 @@ public class PostgresJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest
   @Test
   public void canInsertIntoTableWithCatalogAndSchema() throws Exception {
     createTableWithCatalogAndSchema();
-    sharedRegionSetup("id", DB_NAME, SCHEMA_NAME);
+    sharedRegionSetup("id", DEFAULT_DB_NAME, SCHEMA_NAME);
     employees.put("1", pdx1);
     employees.put("2", pdx2);
 
     ResultSet resultSet =
-        statement.executeQuery("select * from " + DB_NAME + '.' + SCHEMA_NAME + '.'
+        statement.executeQuery("select * from " + DEFAULT_DB_NAME + '.' + SCHEMA_NAME + '.'
             + REGION_TABLE_NAME + " order by id asc");
     assertRecordMatchesEmployee(resultSet, "1", employee1);
     assertRecordMatchesEmployee(resultSet, "2", employee2);
@@ -92,12 +95,12 @@ public class PostgresJdbcWriterIntegrationTest extends JdbcWriterIntegrationTest
   @Test
   public void canUpdateTableWithCatalogAndSchema() throws Exception {
     createTableWithCatalogAndSchema();
-    sharedRegionSetup("id", DB_NAME, SCHEMA_NAME);
+    sharedRegionSetup("id", DEFAULT_DB_NAME, SCHEMA_NAME);
     employees.put("1", pdx1);
     employees.put("1", pdx2);
 
     ResultSet resultSet =
-        statement.executeQuery("select * from " + DB_NAME + '.' + SCHEMA_NAME + '.'
+        statement.executeQuery("select * from " + DEFAULT_DB_NAME + '.' + SCHEMA_NAME + '.'
             + REGION_TABLE_NAME + " order by id asc");
     assertRecordMatchesEmployee(resultSet, "1", employee2);
     assertThat(resultSet.next()).isFalse();
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/TestDataSourceFactory.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/TestDataSourceFactory.java
index 85842bf..6d4ab0a 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/TestDataSourceFactory.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/TestDataSourceFactory.java
@@ -16,6 +16,8 @@
  */
 package org.apache.geode.connectors.jdbc;
 
+import java.util.function.Supplier;
+
 import javax.sql.DataSource;
 
 import com.zaxxer.hikari.HikariDataSource;
@@ -23,10 +25,10 @@ import com.zaxxer.hikari.HikariDataSource;
 import org.apache.geode.connectors.jdbc.internal.SqlHandler.DataSourceFactory;
 
 public class TestDataSourceFactory implements DataSourceFactory {
-  private final String url;
+  private final Supplier<String> url;
   private HikariDataSource dataSource;
 
-  public TestDataSourceFactory(String url) {
+  public TestDataSourceFactory(Supplier<String> url) {
     this.url = url;
   }
 
@@ -34,7 +36,7 @@ public class TestDataSourceFactory implements DataSourceFactory {
   public DataSource getDataSource(String dataSourceName) {
     if (dataSource == null) {
       dataSource = new HikariDataSource();
-      dataSource.setJdbcUrl(url);
+      dataSource.setJdbcUrl(url.get());
     }
     return dataSource;
   }
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/MySqlTableMetaDataManagerIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/MySqlTableMetaDataManagerIntegrationTest.java
index 2017d82..b4b8fa7 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/MySqlTableMetaDataManagerIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/MySqlTableMetaDataManagerIntegrationTest.java
@@ -27,11 +27,11 @@ import org.apache.geode.connectors.jdbc.test.junit.rules.MySqlConnectionRule;
 public class MySqlTableMetaDataManagerIntegrationTest extends TableMetaDataManagerIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      MySqlTableMetaDataManagerIntegrationTest.class.getResource("mysql.yml");
+      MySqlTableMetaDataManagerIntegrationTest.class.getResource("/mysql.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new MySqlConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(3306).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new MySqlConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/PostgresTableMetaDataManagerIntegrationTest.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/PostgresTableMetaDataManagerIntegrationTest.java
index be53014..538a288 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/PostgresTableMetaDataManagerIntegrationTest.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/internal/PostgresTableMetaDataManagerIntegrationTest.java
@@ -34,11 +34,11 @@ public class PostgresTableMetaDataManagerIntegrationTest
     extends TableMetaDataManagerIntegrationTest {
 
   private static final URL COMPOSE_RESOURCE_PATH =
-      PostgresTableMetaDataManagerIntegrationTest.class.getResource("postgres.yml");
+      PostgresTableMetaDataManagerIntegrationTest.class.getResource("/postgres.yml");
 
   @ClassRule
-  public static DatabaseConnectionRule dbRule = new PostgresConnectionRule.Builder()
-      .file(COMPOSE_RESOURCE_PATH.getPath()).serviceName("db").port(5432).database(DB_NAME).build();
+  public static DatabaseConnectionRule dbRule =
+      new PostgresConnectionRule.Builder().file(COMPOSE_RESOURCE_PATH.getPath()).build();
 
   @Override
   public Connection getConnection() throws SQLException {
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/MySqlConnectionRule.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/MySqlConnectionRule.java
index 850cbc9..f8346cc 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/MySqlConnectionRule.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/MySqlConnectionRule.java
@@ -21,18 +21,17 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 
-import com.palantir.docker.compose.DockerComposeRule;
-
 public class MySqlConnectionRule extends SqlDatabaseConnectionRule {
+
+  private static final int MYSQL_PORT = 3306;
+
   private static final String CREATE_DB_CONNECTION_STRING =
-      "jdbc:mysql://$HOST:$EXTERNAL_PORT?user=root&useSSL=false";
+      "jdbc:mysql://%s:%d?user=root&useSSL=false";
 
-  private static final String CONNECTION_STRING =
-      "jdbc:mysql://$HOST:$EXTERNAL_PORT/%s?user=root&useSSL=false";
+  private static final String CONNECTION_STRING = "jdbc:mysql://%s:%d/%s?user=root&useSSL=false";
 
-  protected MySqlConnectionRule(DockerComposeRule dockerRule, String serviceName, int port,
-      String dbName) {
-    super(dockerRule, serviceName, port, dbName);
+  protected MySqlConnectionRule(String composeFile, String serviceName, int port, String dbName) {
+    super(composeFile, serviceName, port, dbName);
   }
 
   @Override
@@ -50,23 +49,23 @@ public class MySqlConnectionRule extends SqlDatabaseConnectionRule {
 
   @Override
   public String getConnectionUrl() {
-    return getDockerPort().inFormat(String.format(CONNECTION_STRING, getDbName()));
+    return String.format(CONNECTION_STRING, "localhost", getDockerPort(), getDbName());
   }
 
 
   public String getCreateDbConnectionUrl() {
-    return getDockerPort().inFormat(CREATE_DB_CONNECTION_STRING);
+    return String.format(CREATE_DB_CONNECTION_STRING, "localhost", getDockerPort());
   }
 
   public static class Builder extends SqlDatabaseConnectionRule.Builder {
 
     public Builder() {
-      super();
+      super(MYSQL_PORT, DEFAULT_SERVICE_NAME, DEFAULT_DB_NAME);
     }
 
     @Override
     public MySqlConnectionRule build() {
-      return new MySqlConnectionRule(createDockerRule(), getServiceName(), getPort(), getDbName());
+      return new MySqlConnectionRule(getComposeFile(), getServiceName(), getPort(), getDbName());
     }
   }
 }
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/PostgresConnectionRule.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/PostgresConnectionRule.java
index cbf515e..17c985d 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/PostgresConnectionRule.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/PostgresConnectionRule.java
@@ -14,31 +14,31 @@
  */
 package org.apache.geode.connectors.jdbc.test.junit.rules;
 
-import com.palantir.docker.compose.DockerComposeRule;
-
 public class PostgresConnectionRule extends SqlDatabaseConnectionRule {
-  private static final String CONNECTION_STRING =
-      "jdbc:postgresql://$HOST:$EXTERNAL_PORT/%s?user=postgres";
 
-  protected PostgresConnectionRule(DockerComposeRule dockerRule, String serviceName, int port,
+  private static final int POSTGRES_PORT = 5432;
+
+  private static final String CONNECTION_STRING = "jdbc:postgresql://%s:%d/%s?user=postgres";
+
+  protected PostgresConnectionRule(String composeFile, String serviceName, int port,
       String dbName) {
-    super(dockerRule, serviceName, port, dbName);
+    super(composeFile, serviceName, port, dbName);
   }
 
   @Override
   public String getConnectionUrl() {
-    return getDockerPort().inFormat(String.format(CONNECTION_STRING, getDbName()));
+    return String.format(CONNECTION_STRING, "localhost", getDockerPort(), getDbName());
   }
 
   public static class Builder extends SqlDatabaseConnectionRule.Builder {
 
     public Builder() {
-      super();
+      super(POSTGRES_PORT, DEFAULT_SERVICE_NAME, DEFAULT_DB_NAME);
     }
 
     @Override
     public PostgresConnectionRule build() {
-      return new PostgresConnectionRule(createDockerRule(), getServiceName(), getPort(),
+      return new PostgresConnectionRule(getComposeFile(), getServiceName(), getPort(),
           getDbName());
     }
   }
diff --git a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/SqlDatabaseConnectionRule.java b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/SqlDatabaseConnectionRule.java
index 1010657..39f999f 100644
--- a/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/SqlDatabaseConnectionRule.java
+++ b/geode-connectors/src/acceptanceTest/java/org/apache/geode/connectors/jdbc/test/junit/rules/SqlDatabaseConnectionRule.java
@@ -17,44 +17,58 @@ package org.apache.geode.connectors.jdbc.test.junit.rules;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
-import java.io.IOException;
+import java.io.File;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 
-import com.palantir.docker.compose.DockerComposeRule;
-import com.palantir.docker.compose.connection.DockerPort;
-import com.palantir.docker.compose.connection.waiting.HealthChecks;
 import org.junit.rules.ExternalResource;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+import org.testcontainers.containers.DockerComposeContainer;
 
 public abstract class SqlDatabaseConnectionRule extends ExternalResource
     implements DatabaseConnectionRule {
 
-  private final DockerComposeRule dockerRule;
+  public static final String DEFAULT_DB_NAME = "test";
+  protected static final String DEFAULT_SERVICE_NAME = "db";
+  private DockerComposeContainer<?> dbContainer;
+  private final String composeFile;
   private final String serviceName;
   private final int port;
   private final String dbName;
 
-  protected SqlDatabaseConnectionRule(DockerComposeRule dockerRule, String serviceName, int port,
+  protected SqlDatabaseConnectionRule(String composeFile, String serviceName, int port,
       String dbName) {
-    this.dockerRule = dockerRule;
+    this.composeFile = composeFile;
     this.serviceName = serviceName;
     this.port = port;
     this.dbName = dbName;
   }
 
   @Override
-  public void before() throws IOException, InterruptedException {
-    dockerRule.before();
+  public Statement apply(Statement base, Description description) {
+    Statement dbStatement = new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+
+        dbContainer = new DockerComposeContainer<>("db", new File(composeFile))
+            .withExposedService(serviceName, port);
+        dbContainer.start();
+
+        try {
+          base.evaluate(); // run the test
+        } finally {
+          dbContainer.stop();
+        }
+      }
+    };
+
+    return dbStatement;
   }
 
-  @Override
-  public void after() {
-    dockerRule.after();
-  }
-
-  protected DockerPort getDockerPort() {
-    return dockerRule.containers().container(serviceName).port(port);
+  protected Integer getDockerPort() {
+    return dbContainer.getServicePort(serviceName, port);
   }
 
   protected String getDbName() {
@@ -79,6 +93,12 @@ public abstract class SqlDatabaseConnectionRule extends ExternalResource
     private int port;
     private String dbName;
 
+    protected Builder(int port, String serviceName, String dbName) {
+      this.port = port;
+      this.serviceName = serviceName;
+      this.dbName = dbName;
+    }
+
     public abstract SqlDatabaseConnectionRule build();
 
     public Builder file(String filePath) {
@@ -101,6 +121,10 @@ public abstract class SqlDatabaseConnectionRule extends ExternalResource
       return this;
     }
 
+    protected String getComposeFile() {
+      return filePath;
+    }
+
     protected String getDbName() {
       return dbName;
     }
@@ -113,11 +137,6 @@ public abstract class SqlDatabaseConnectionRule extends ExternalResource
       return port;
     }
 
-    protected DockerComposeRule createDockerRule() {
-      return DockerComposeRule.builder().file(filePath)
-          .waitingForService(serviceName, HealthChecks.toHaveAllPortsOpen()).build();
-    }
-
   }
 
 }
diff --git a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/mysql.yml b/geode-connectors/src/acceptanceTest/resources/mysql.yml
similarity index 92%
rename from geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/mysql.yml
rename to geode-connectors/src/acceptanceTest/resources/mysql.yml
index 6b127df..2163918 100644
--- a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/mysql.yml
+++ b/geode-connectors/src/acceptanceTest/resources/mysql.yml
@@ -14,15 +14,13 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-version: '2.0'
+version: '3.0'
 
 services:
 
   db:
     image: mysql:5.7
     restart: always
-    ports:
-        - 3306
     environment:
       MYSQL_ALLOW_EMPTY_PASSWORD: "true"
-      #MYSQL_ROOT_PASSWORD: "secret"
\ No newline at end of file
+      #MYSQL_ROOT_PASSWORD: "secret"
diff --git a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/internal/mysql.yml b/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/internal/mysql.yml
deleted file mode 100644
index 6b127df..0000000
--- a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/internal/mysql.yml
+++ /dev/null
@@ -1,28 +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.
-#
-version: '2.0'
-
-services:
-
-  db:
-    image: mysql:5.7
-    restart: always
-    ports:
-        - 3306
-    environment:
-      MYSQL_ALLOW_EMPTY_PASSWORD: "true"
-      #MYSQL_ROOT_PASSWORD: "secret"
\ No newline at end of file
diff --git a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/internal/postgres.yml b/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/internal/postgres.yml
deleted file mode 100644
index 00d95db..0000000
--- a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/internal/postgres.yml
+++ /dev/null
@@ -1,28 +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.
-#
-version: '2.0'
-
-services:
-
-  db:
-    image: postgres
-    restart: always
-    ports:
-      - 5432
-    environment:
-      POSTGRES_DB: test
-      POSTGRES_HOST_AUTH_METHOD: trust
diff --git a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/postgres.yml b/geode-connectors/src/acceptanceTest/resources/postgres.yml
similarity index 95%
rename from geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/postgres.yml
rename to geode-connectors/src/acceptanceTest/resources/postgres.yml
index 00d95db..71f973c 100644
--- a/geode-connectors/src/acceptanceTest/resources/org/apache/geode/connectors/jdbc/postgres.yml
+++ b/geode-connectors/src/acceptanceTest/resources/postgres.yml
@@ -14,15 +14,13 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-version: '2.0'
+version: '3.0'
 
 services:
 
   db:
     image: postgres
     restart: always
-    ports:
-      - 5432
     environment:
       POSTGRES_DB: test
       POSTGRES_HOST_AUTH_METHOD: trust