You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2018/11/20 15:37:30 UTC

[geode] branch develop updated: GEODE-4110: improve junit ClientCacheRule (#2872)

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

jinmeiliao pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 222701a  GEODE-4110: improve junit ClientCacheRule (#2872)
222701a is described below

commit 222701ad7511841b1ba0033fc51c0ac6b044efe8
Author: jinmeiliao <ji...@pivotal.io>
AuthorDate: Tue Nov 20 07:37:19 2018 -0800

    GEODE-4110: improve junit ClientCacheRule (#2872)
    
    * refactor some multiUser tests to use the ClientCacheRule
    * add more convenience methods in ClientCacheRule
    * clean up the ClusterStarterRule.startClient interface
    * deprecate the SecurityTestUtils to favor using rules
---
 .../web/controllers/RestAPIsWithSSLDUnitTest.java  |   4 +-
 .../geode/connectors/jdbc/JdbcDistributedTest.java |   3 +-
 .../management/ClientHealthStatsDUnitTest.java     |  19 +-
 .../internal/security/MultiClientDUnitTest.java    |  20 +-
 .../security/MultiUserAuthenticationDUnitTest.java |  47 +--
 .../apache/geode/security/ClientAuthDUnitTest.java |  38 +-
 ...tAuthorizationLegacyConfigurationDUnitTest.java |  74 ++--
 ...aAuthorizationUsingLegacySecurityDUnitTest.java |  16 +-
 ...onUsingLegacySecurityWithFailoverDUnitTest.java |   8 +-
 .../ClientDestroyInvalidateAuthDUnitTest.java      |   9 +-
 .../ClientExecuteFunctionAuthDUnitTest.java        |   9 +-
 .../cache/query/dunit/QueryMonitorDUnitTest.java   |  35 +-
 .../geode/security/ClientCQAuthDUnitTest.java      |  49 ++-
 .../geode/security/MultiUserAPIDUnitTest.java      | 407 ++++++---------------
 ...tartupRuleCanSpecifyOlderVersionsDUnitTest.java |   4 +-
 .../apache/geode/security/SecurityTestUtil.java    |   7 +
 .../geode/test/dunit/rules/ClusterStartupRule.java |  56 +--
 .../apache/geode/test/dunit/rules/MemberVM.java    |   2 +-
 .../geode/test/junit/rules/ClientCacheRule.java    |  80 +++-
 .../geode/test/junit/rules/MemberStarterRule.java  |  39 ++
 .../geode/test/junit/rules/ServerStarterRule.java  |  33 --
 21 files changed, 425 insertions(+), 534 deletions(-)

diff --git a/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java b/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
index 8f813a4..e876f6c 100644
--- a/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
+++ b/geode-assembly/src/distributedTest/java/org/apache/geode/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
@@ -128,8 +128,8 @@ public class RestAPIsWithSSLDUnitTest {
         .withProperties(sslProperties)
         .withConnectionToLocator(locatorPort)
         .withRegion(RegionShortcut.REPLICATE, PEOPLE_REGION_NAME));
-    client = cluster.startClientVM(1,
-        c -> c.addPoolLocator("localhost", locatorPort).setPdxReadSerialized(true));
+    client = cluster.startClientVM(1, c -> c.withLocatorConnection(locatorPort)
+        .withCacheSetup(cf -> cf.setPdxReadSerialized(true)));
 
     client.invoke(() -> {
       ClientCache clientCache = ClusterStartupRule.getClientCache();
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 94370e9..94fa59d 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
@@ -28,7 +28,6 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Date;
-import java.util.Properties;
 
 import org.junit.After;
 import org.junit.Before;
@@ -605,7 +604,7 @@ public abstract class JdbcDistributedTest implements Serializable {
       cf.setPdxSerializer(
           new ReflectionBasedAutoSerializer(ClassWithSupportedPdxFields.class.getName()));
     };
-    return startupRule.startClientVM(2, new Properties(), cacheSetup);
+    return startupRule.startClientVM(2, c -> c.withCacheSetup(cacheSetup));
   }
 
   private void createClientRegion(ClientVM client) {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/ClientHealthStatsDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/ClientHealthStatsDUnitTest.java
index bf21c2a..ea7a642 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/ClientHealthStatsDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/ClientHealthStatsDUnitTest.java
@@ -21,6 +21,7 @@ import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.Serializable;
+import java.util.Properties;
 
 import org.junit.Before;
 import org.junit.Rule;
@@ -62,8 +63,12 @@ public class ClientHealthStatsDUnitTest implements Serializable {
 
   @Test
   public void testClientHealthStats_SubscriptionEnabled() throws Exception {
-    client1 = cluster.startClientVM(2, true, server.getPort());
-    client2 = cluster.startClientVM(3, true, server.getPort());
+    client1 =
+        cluster.startClientVM(2,
+            c1 -> c1.withPoolSubscription(true).withServerConnection(server.getPort()));
+    client2 =
+        cluster.startClientVM(3,
+            c -> c.withPoolSubscription(true).withServerConnection(server.getPort()));
 
     VMProvider.invokeInEveryMember(() -> {
       ClientRegionFactory<String, String> regionFactory =
@@ -80,8 +85,12 @@ public class ClientHealthStatsDUnitTest implements Serializable {
 
   @Test
   public void testClientHealthStats_SubscriptionDisabled() throws Exception {
-    client1 = cluster.startClientVM(2, false, server.getPort());
-    client2 = cluster.startClientVM(3, false, server.getPort());
+    client1 =
+        cluster.startClientVM(2,
+            c1 -> c1.withPoolSubscription(false).withServerConnection(server.getPort()));
+    client2 =
+        cluster.startClientVM(3,
+            c -> c.withPoolSubscription(false).withServerConnection(server.getPort()));
     VMProvider.invokeInEveryMember(() -> {
       ClientRegionFactory<String, String> regionFactory =
           ClusterStartupRule.getClientCache()
@@ -171,7 +180,7 @@ public class ClientHealthStatsDUnitTest implements Serializable {
   }
 
   private ClientVM createDurableClient(int index) throws Exception {
-    ClientVM client = cluster.startClientVM(index, ccf -> {
+    ClientVM client = cluster.startClientVM(index, new Properties(), ccf -> {
       ccf.setPoolSubscriptionEnabled(true);
       ccf.addPoolServer("localhost", server.getPort());
       ccf.set(DURABLE_CLIENT_ID, "client" + index);
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java
index a966189..d2b7bc8 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiClientDUnitTest.java
@@ -77,12 +77,20 @@ public class MultiClientDUnitTest {
       cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
     }, server1, server2);
 
-    client3 = lsRule.startClientVM(3, "data", "data", false, server1.getPort(), server2.getPort());
-    client4 = lsRule.startClientVM(4, "stranger", "stranger", false, server1.getPort(),
-        server2.getPort());
-    client5 = lsRule.startClientVM(5, "data", "data", false, server1.getPort(), server2.getPort());
-    client6 = lsRule.startClientVM(6, "dataWithWrongPswd", "data", false, server1.getPort(),
-        server2.getPort());
+    int server1Port = server1.getPort();
+    int server2Port = server2.getPort();
+    client3 = lsRule.startClientVM(3, c -> c.withCredential("data", "data")
+        .withPoolSubscription(false)
+        .withServerConnection(server1Port, server2Port));
+    client4 = lsRule.startClientVM(4, c -> c.withCredential("stranger", "stranger")
+        .withPoolSubscription(false)
+        .withServerConnection(server1Port, server2Port));
+    client5 = lsRule.startClientVM(5, c -> c.withCredential("data", "data")
+        .withPoolSubscription(false)
+        .withServerConnection(server1Port, server2Port));
+    client6 = lsRule.startClientVM(6, c -> c.withCredential("dataWithWrongPswd", "data")
+        .withPoolSubscription(false)
+        .withServerConnection(server1Port, server2Port));
   }
 
   @Test
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java
index d58c222..6ad4b61 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/management/internal/security/MultiUserAuthenticationDUnitTest.java
@@ -15,31 +15,27 @@
 
 package org.apache.geode.management.internal.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 import java.util.Properties;
 
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionService;
 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.ServerOperationException;
 import org.apache.geode.examples.SimpleSecurityManager;
-import org.apache.geode.security.templates.UserPasswordAuthInit;
 import org.apache.geode.test.dunit.IgnoredException;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.rules.ClientCacheRule;
 import org.apache.geode.test.junit.rules.GfshCommandRule;
 
 @Category({SecurityTest.class})
@@ -51,23 +47,25 @@ public class MultiUserAuthenticationDUnitTest {
   @ClassRule
   public static ClusterStartupRule lsRule = new ClusterStartupRule();
 
+  @Rule
+  public ClientCacheRule client = new ClientCacheRule();
+
   @ClassRule
   public static GfshCommandRule gfsh = new GfshCommandRule();
 
-  private static MemberVM locator, server1, server2;
+  private static MemberVM locator;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
     IgnoredException.addIgnoredException("org.apache.geode.security.AuthenticationFailedException");
-    Properties locatorProps = new Properties();
-    locatorProps.setProperty(SECURITY_MANAGER, SimpleSecurityManager.class.getCanonicalName());
-    locator = lsRule.startLocatorVM(0, locatorProps);
+
+    locator = lsRule.startLocatorVM(0, l -> l.withSecurityManager(SimpleSecurityManager.class));
 
     Properties serverProps = new Properties();
     serverProps.setProperty("security-username", "cluster");
     serverProps.setProperty("security-password", "cluster");
-    server1 = lsRule.startServerVM(1, serverProps, locator.getPort());
-    server2 = lsRule.startServerVM(2, serverProps, locator.getPort());
+    lsRule.startServerVM(1, serverProps, locator.getPort());
+    lsRule.startServerVM(2, serverProps, locator.getPort());
 
     // create region and put in some values
     gfsh.connectAndVerify(locator);
@@ -78,14 +76,14 @@ public class MultiUserAuthenticationDUnitTest {
   public void multiAuthenticatedView() throws Exception {
     int locatorPort = locator.getPort();
     for (int i = 0; i < SESSION_COUNT; i++) {
-      ClientCache cache = new ClientCacheFactory(getClientCacheProperties("stranger", "stranger"))
-          .setPoolSubscriptionEnabled(true).setPoolMultiuserAuthentication(true)
-          .addPoolLocator("localhost", locatorPort).create();
+      ClientCache cache = client.withCredential("stranger", "stranger")
+          .withCacheSetup(f -> f.setPoolSubscriptionEnabled(true)
+              .setPoolMultiuserAuthentication(true)
+              .addPoolLocator("localhost", locatorPort))
+          .createCache();
 
-      RegionService regionService1 =
-          cache.createAuthenticatedView(getClientCacheProperties("data", "data"));
-      RegionService regionService2 =
-          cache.createAuthenticatedView(getClientCacheProperties("cluster", "cluster"));
+      RegionService regionService1 = client.createAuthenticatedView("data", "data");
+      RegionService regionService2 = client.createAuthenticatedView("cluster", "cluster");
 
       cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create("region");
 
@@ -102,15 +100,4 @@ public class MultiUserAuthenticationDUnitTest {
       cache.close();
     }
   }
-
-  private static Properties getClientCacheProperties(String username, String password) {
-    Properties props = new Properties();
-    props.setProperty(UserPasswordAuthInit.USER_NAME, username);
-    props.setProperty(UserPasswordAuthInit.PASSWORD, password);
-    props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
-    props.setProperty(LOCATORS, "");
-    props.setProperty(MCAST_PORT, "0");
-    return props;
-  }
-
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthDUnitTest.java
index e3298d7..76398c3 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthDUnitTest.java
@@ -67,10 +67,11 @@ public class ClientAuthDUnitTest {
   @Test
   public void authWithCorrectPasswordShouldPass() throws Exception {
     int serverPort = server.getPort();
-    ClientVM clientVM = lsRule.startClientVM(0, getClientAuthProperties("data", "data"), ccf -> {
-      ccf.setPoolSubscriptionEnabled(true);
-      ccf.addPoolServer("localhost", serverPort);
-    }, clientVersion);
+    ClientVM clientVM = lsRule.startClientVM(0, clientVersion,
+        getClientAuthProperties("data", "data"), ccf -> {
+          ccf.setPoolSubscriptionEnabled(true);
+          ccf.addPoolServer("localhost", serverPort);
+        });
 
     clientVM.invoke(() -> {
       ClientCache clientCache = ClusterStartupRule.getClientCache();
@@ -90,19 +91,21 @@ public class ClientAuthDUnitTest {
     // authentication error will happen at this step.
     if (Arrays.asList("100", "110", "111", "120", "130", "140").contains(clientVersion)) {
       assertThatThrownBy(
-          () -> lsRule.startClientVM(0, getClientAuthProperties("test", "invalidPassword"), ccf -> {
-            ccf.setPoolSubscriptionEnabled(true);
-            ccf.addPoolServer("localhost", serverPort);
-          }, clientVersion))
-              .isInstanceOf(AuthenticationFailedException.class);
+          () -> lsRule.startClientVM(0, clientVersion,
+              getClientAuthProperties("test", "invalidPassword"), ccf -> {
+                ccf.setPoolSubscriptionEnabled(true);
+                ccf.addPoolServer("localhost", serverPort);
+              }))
+                  .isInstanceOf(AuthenticationFailedException.class);
       return;
     }
 
     ClientVM clientVM =
-        lsRule.startClientVM(0, getClientAuthProperties("test", "invalidPassword"), ccf -> {
-          ccf.setPoolSubscriptionEnabled(true);
-          ccf.addPoolServer("localhost", serverPort);
-        }, clientVersion);
+        lsRule.startClientVM(0, clientVersion, getClientAuthProperties("test", "invalidPassword"),
+            ccf -> {
+              ccf.setPoolSubscriptionEnabled(true);
+              ccf.addPoolServer("localhost", serverPort);
+            });
 
     clientVM.invoke(() -> {
       ClientCache clientCache = ClusterStartupRule.getClientCache();
@@ -118,10 +121,11 @@ public class ClientAuthDUnitTest {
     int serverPort = server.getPort();
     IgnoredException.addIgnoredException(AuthenticationFailedException.class.getName());
     ClientVM clientVM =
-        lsRule.startClientVM(0, getClientAuthProperties("test", "invalidPassword"), ccf -> {
-          ccf.setPoolSubscriptionEnabled(false);
-          ccf.addPoolServer("localhost", serverPort);
-        }, clientVersion);
+        lsRule.startClientVM(0, clientVersion, getClientAuthProperties("test", "invalidPassword"),
+            ccf -> {
+              ccf.setPoolSubscriptionEnabled(false);
+              ccf.addPoolServer("localhost", serverPort);
+            });
 
     clientVM.invoke(() -> {
       ClientCache clientCache = ClusterStartupRule.getClientCache();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthorizationLegacyConfigurationDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthorizationLegacyConfigurationDUnitTest.java
index 7dae157..d8caf8b 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthorizationLegacyConfigurationDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientAuthorizationLegacyConfigurationDUnitTest.java
@@ -33,13 +33,11 @@ import org.junit.runners.Parameterized;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.security.templates.SimpleAccessController;
-import org.apache.geode.security.templates.SimpleAuthenticator;
 import org.apache.geode.security.templates.UserPasswordAuthInit;
 import org.apache.geode.test.dunit.rules.ClientVM;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
@@ -86,39 +84,20 @@ public class ClientAuthorizationLegacyConfigurationDUnitTest {
 
   @Test
   public void everythingFailsWithInvalidAuthenticator() throws Exception {
-    Properties clusterProps = new Properties();
-    clusterProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR,
-        "org.apache.geode.no.such.authenticator.create");
-    clusterProps.setProperty(SECURITY_CLIENT_ACCESSOR,
-        SimpleAccessController.class.getName() + ".create");
-    clusterProps.setProperty(UserPasswordAuthInit.USER_NAME, "cluster,data");
-    clusterProps.setProperty(UserPasswordAuthInit.PASSWORD, "cluster,data");
-    clusterProps.setProperty(SECURITY_CLIENT_AUTH_INIT,
-        UserPasswordAuthInit.class.getCanonicalName() + ".create");
+    Properties clusterProps = getClusterProperties();
 
     locator = csRule.startLocatorVM(0, clusterProps);
     server = csRule.startServerVM(1, clusterProps, locator.getPort());
     server.invoke(() -> {
       Cache cache = ClusterStartupRule.getCache();
-      RegionFactory<String, String> rf = cache.createRegionFactory(RegionShortcut.PARTITION);
-      Region<String, String> region = rf.create(regionName);
+      Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(regionName);
       region.put(initKey, initValue);
     });
 
-    Properties clientProps = new Properties();
-    clusterProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR,
-        "org.apache.geode.no.such.authenticator.create");
-    clusterProps.setProperty(SECURITY_CLIENT_ACCESSOR,
-        SimpleAccessController.class.getName() + ".create");
-    clientProps.setProperty(UserPasswordAuthInit.USER_NAME, "data");
-    clientProps.setProperty(UserPasswordAuthInit.PASSWORD, "data");
-    clientProps.setProperty(SECURITY_CLIENT_AUTH_INIT,
-        UserPasswordAuthInit.class.getCanonicalName() + ".create");
-
     int locatorPort = locator.getPort();
 
-    ClientVM client = csRule.startClientVM(2, clientProps, cf -> cf
-        .addPoolLocator("localhost", locatorPort), clientVersion);
+    ClientVM client = csRule.startClientVM(2, clientVersion, c -> c.withCredential("data", "data")
+        .withLocatorConnection(locatorPort));
 
     client.invoke(() -> {
       ClientCache cache = ClusterStartupRule.getClientCache();
@@ -148,44 +127,24 @@ public class ClientAuthorizationLegacyConfigurationDUnitTest {
 
   @Test
   public void everythingFailsWithInvalidAccessor() throws Exception {
-    Properties clusterProps = new Properties();
-    clusterProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR,
-        SimpleAuthenticator.class.getCanonicalName() + ".create");
-    clusterProps.setProperty(SECURITY_CLIENT_ACCESSOR, "org.apache.geode.no.such.accessor.create");
-    // give cluster members super-user permissions for ease of testing / RMI invocation
-    clusterProps.setProperty(UserPasswordAuthInit.USER_NAME, "cluster,data");
-    clusterProps.setProperty(UserPasswordAuthInit.PASSWORD, "cluster,data");
-    clusterProps.setProperty(SECURITY_CLIENT_AUTH_INIT,
-        UserPasswordAuthInit.class.getCanonicalName() + ".create");
+    Properties clusterProps = getClusterProperties();
 
     locator = csRule.startLocatorVM(0, clusterProps);
     server = csRule.startServerVM(1, clusterProps, locator.getPort());
     server.invoke(() -> {
       Cache cache = ClusterStartupRule.getCache();
-      RegionFactory<String, String> rf = cache.createRegionFactory(RegionShortcut.PARTITION);
-      Region<String, String> region = rf.create(regionName);
+      Region region = cache.createRegionFactory(RegionShortcut.PARTITION).create(regionName);
       region.put(initKey, initValue);
     });
 
-    Properties clientProps = new Properties();
-    clientProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR,
-        SimpleAuthenticator.class.getCanonicalName() + ".create");
-    clientProps.setProperty(SECURITY_CLIENT_ACCESSOR, "org.apache.geode.no.such.accessor.create");
-    // give cluster members super-user permissions for ease of testing / RMI invocation
-    clientProps.setProperty(UserPasswordAuthInit.USER_NAME, "data");
-    clientProps.setProperty(UserPasswordAuthInit.PASSWORD, "data");
-    clientProps.setProperty(SECURITY_CLIENT_AUTH_INIT,
-        UserPasswordAuthInit.class.getCanonicalName() + ".create");
-
     int locatorPort = locator.getPort();
 
-    ClientVM client = csRule.startClientVM(2, clientProps, cf -> cf
-        .addPoolLocator("localhost", locatorPort), clientVersion);
+    ClientVM client = csRule.startClientVM(2, clientVersion,
+        c -> c.withCredential("data", "data").withLocatorConnection(locatorPort));
     client.invoke(() -> {
       ClientCache cache = ClusterStartupRule.getClientCache();
-      ClientRegionFactory<String, String> rf =
-          cache.createClientRegionFactory(ClientRegionShortcut.PROXY);
-      Region<String, String> region = rf.create(regionName);
+      Region region =
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(regionName);
 
       // Assert that everything is horrible
       assertThatThrownBy(() -> region.get(initKey))
@@ -207,5 +166,16 @@ public class ClientAuthorizationLegacyConfigurationDUnitTest {
     });
   }
 
-
+  private Properties getClusterProperties() {
+    Properties clusterProps = new Properties();
+    clusterProps.setProperty(SECURITY_CLIENT_AUTHENTICATOR,
+        "org.apache.geode.no.such.authenticator.create");
+    clusterProps.setProperty(SECURITY_CLIENT_ACCESSOR,
+        SimpleAccessController.class.getName() + ".create");
+    clusterProps.setProperty(UserPasswordAuthInit.USER_NAME, "cluster,data");
+    clusterProps.setProperty(UserPasswordAuthInit.PASSWORD, "cluster,data");
+    clusterProps.setProperty(SECURITY_CLIENT_AUTH_INIT,
+        UserPasswordAuthInit.class.getCanonicalName() + ".create");
+    return clusterProps;
+  }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityDUnitTest.java
index 6f0c589..04c9f5e 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityDUnitTest.java
@@ -119,8 +119,8 @@ public class ClientDataAuthorizationUsingLegacySecurityDUnitTest {
     Properties props = getVMPropertiesWithPermission("dataWrite");
     int locatorPort = locator.getPort();
 
-    ClientVM clientVM = csRule.startClientVM(2, props, cf -> cf
-        .addPoolLocator("localhost", locatorPort), clientVersion);
+    ClientVM clientVM = csRule.startClientVM(2, clientVersion, props, cf -> cf
+        .addPoolLocator("localhost", locatorPort));
 
     // Client adds data
     clientVM.invoke(() -> {
@@ -152,8 +152,8 @@ public class ClientDataAuthorizationUsingLegacySecurityDUnitTest {
     }
     int locatorPort = locator.getPort();
 
-    ClientVM client = csRule.startClientVM(2, props, cf -> cf
-        .addPoolLocator("localhost", locatorPort), clientVersion);
+    ClientVM client = csRule.startClientVM(2, clientVersion, props, cf -> cf
+        .addPoolLocator("localhost", locatorPort));
 
     // Client cannot get through any avenue
     client.invoke(() -> {
@@ -178,8 +178,8 @@ public class ClientDataAuthorizationUsingLegacySecurityDUnitTest {
     Properties props = getVMPropertiesWithPermission("dataRead");
     int locatorPort = locator.getPort();
 
-    ClientVM client = csRule.startClientVM(2, props, cf -> cf
-        .addPoolLocator("localhost", locatorPort), clientVersion);
+    ClientVM client = csRule.startClientVM(2, clientVersion, props, cf -> cf
+        .addPoolLocator("localhost", locatorPort));
 
     // Add some values for the client to get
     server.invoke(() -> {
@@ -214,8 +214,8 @@ public class ClientDataAuthorizationUsingLegacySecurityDUnitTest {
 
     int locatorPort = locator.getPort();
 
-    ClientVM clientVM = csRule.startClientVM(2, props, cf -> cf
-        .addPoolLocator("localhost", locatorPort), clientVersion);
+    ClientVM clientVM = csRule.startClientVM(2, clientVersion, props, cf -> cf
+        .addPoolLocator("localhost", locatorPort));
 
     clientVM.invoke(() -> {
       ClientCache cache = ClusterStartupRule.getClientCache();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityWithFailoverDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityWithFailoverDUnitTest.java
index 560b767..593575c 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityWithFailoverDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDataAuthorizationUsingLegacySecurityWithFailoverDUnitTest.java
@@ -326,9 +326,9 @@ public class ClientDataAuthorizationUsingLegacySecurityWithFailoverDUnitTest {
     int server1Port = this.server1.getPort();
     int server2Port = this.server2.getPort();
 
-    ClientVM client1 = csRule.startClientVM(3, props, cf -> cf
+    ClientVM client1 = csRule.startClientVM(3, clientVersion, props, cf -> cf
         .addPoolServer("localhost", server1Port).addPoolServer("localhost", server2Port)
-        .setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(2), clientVersion);
+        .setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(2));
 
     // Initialize cache
     client1.invoke(() -> {
@@ -384,9 +384,9 @@ public class ClientDataAuthorizationUsingLegacySecurityWithFailoverDUnitTest {
           "org.apache.geode.security.templates.UsernamePrincipal");
     }
 
-    ClientVM client = csRule.startClientVM(3, props, cf -> cf
+    ClientVM client = csRule.startClientVM(3, clientVersion, props, cf -> cf
         .addPoolServer("localhost", server1Port).addPoolServer("localhost", server2Port)
-        .setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(2), clientVersion);
+        .setPoolSubscriptionEnabled(true).setPoolSubscriptionRedundancy(2));
 
     // Initialize cache
     client.invoke(() -> {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java
index 0715590..15edfb1 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientDestroyInvalidateAuthDUnitTest.java
@@ -57,7 +57,10 @@ public class ClientDestroyInvalidateAuthDUnitTest {
 
   @Test
   public void testDestroyInvalidate() throws Exception {
-    client1 = lsRule.startClientVM(1, "data", "data", true, server.getPort());
+    int serverPort = server.getPort();
+    client1 = lsRule.startClientVM(1, c1 -> c1.withCredential("data", "data")
+        .withPoolSubscription(true)
+        .withServerConnection(serverPort));
     // Delete one key and invalidate another key with an authorized user.
     client1.invoke(() -> {
       ClientCache cache = ClusterStartupRule.getClientCache();
@@ -76,7 +79,9 @@ public class ClientDestroyInvalidateAuthDUnitTest {
       cache.close();
     });
 
-    client2 = lsRule.startClientVM(2, "dataRead", "dataRead", true, server.getPort());
+    client2 = lsRule.startClientVM(2, c -> c.withCredential("dataRead", "dataRead")
+        .withPoolSubscription(true)
+        .withServerConnection(serverPort));
     // Delete one key and invalidate another key with an unauthorized user.
     client2.invoke(() -> {
       ClientCache cache = ClusterStartupRule.getClientCache();
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java
index 2889d0d..e15f3f4 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/ClientExecuteFunctionAuthDUnitTest.java
@@ -62,8 +62,13 @@ public class ClientExecuteFunctionAuthDUnitTest {
     server.invoke(() -> {
       ClusterStartupRule.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("region");
     });
-    client1 = cluster.startClientVM(1, "dataRead", "dataRead", true, server.getPort());
-    client2 = cluster.startClientVM(2, "dataWrite", "dataWrite", true, server.getPort());
+    int serverPort = server.getPort();
+    client1 = cluster.startClientVM(1, c1 -> c1.withCredential("dataRead", "dataRead")
+        .withPoolSubscription(true)
+        .withServerConnection(serverPort));
+    client2 = cluster.startClientVM(2, c -> c.withCredential("dataWrite", "dataWrite")
+        .withPoolSubscription(true)
+        .withServerConnection(serverPort));
 
     VMProvider.invokeInEveryMember(() -> {
       writeFunction = new WriteFunction();
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
index c3df5ef..e966aed 100644
--- a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
@@ -18,6 +18,7 @@ import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.junit.Assert.fail;
 
 import java.io.File;
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 import org.junit.After;
@@ -102,8 +103,12 @@ public class QueryMonitorDUnitTest {
   @Test
   public void testMultipleClientToOneServer() throws Exception {
     int server1Port = server1.getPort();
-    client3 = cluster.startClientVM(3, true, server1Port);
-    client4 = cluster.startClientVM(4, true, server1Port);
+    client3 =
+        cluster.startClientVM(3,
+            c1 -> c1.withPoolSubscription(true).withServerConnection(server1Port));
+    client4 =
+        cluster.startClientVM(4,
+            c -> c.withPoolSubscription(true).withServerConnection(server1Port));
 
     gfsh.executeAndAssertThat("create region --name=exampleRegion --type=REPLICATE")
         .statusIsSuccess();
@@ -119,7 +124,9 @@ public class QueryMonitorDUnitTest {
   public void testOneClientToMultipleServerOnReplicateRegion() throws Exception {
     int server1Port = server1.getPort();
     int server2Port = server2.getPort();
-    client3 = cluster.startClientVM(3, true, server1Port, server2Port);
+    client3 =
+        cluster.startClientVM(3, c -> c.withPoolSubscription(true)
+            .withServerConnection(new int[] {server1Port, server2Port}));
 
     gfsh.executeAndAssertThat("create region --name=exampleRegion --type=REPLICATE")
         .statusIsSuccess();
@@ -136,8 +143,12 @@ public class QueryMonitorDUnitTest {
     // client3 connects to server1, client4 connects to server2
     int server1Port = server1.getPort();
     int server2Port = server2.getPort();
-    client3 = cluster.startClientVM(3, true, server1Port);
-    client4 = cluster.startClientVM(4, true, server2Port);
+    client3 =
+        cluster.startClientVM(3,
+            c1 -> c1.withPoolSubscription(true).withServerConnection(server1Port));
+    client4 =
+        cluster.startClientVM(4,
+            c -> c.withPoolSubscription(true).withServerConnection(server2Port));
 
     gfsh.executeAndAssertThat("create region --name=exampleRegion --type=PARTITION")
         .statusIsSuccess();
@@ -199,11 +210,11 @@ public class QueryMonitorDUnitTest {
     // client3 connects to server1, client4 connects to server2
     int server1Port = server1.getPort();
     int server2Port = server2.getPort();
-    client3 = cluster.startClientVM(3, ccf -> {
+    client3 = cluster.startClientVM(3, new Properties(), ccf -> {
       configureClientCacheFactory(ccf, server1Port);
     });
 
-    client4 = cluster.startClientVM(4, ccf -> {
+    client4 = cluster.startClientVM(4, new Properties(), ccf -> {
       configureClientCacheFactory(ccf, server2Port);
     });
     client3.invoke(() -> executeQuery());
@@ -232,8 +243,12 @@ public class QueryMonitorDUnitTest {
     // client3 connects to server1, client4 connects to server2
     int server1Port = server1.getPort();
     int server2Port = server2.getPort();
-    client3 = cluster.startClientVM(3, true, server1Port);
-    client4 = cluster.startClientVM(4, true, server2Port);
+    client3 =
+        cluster.startClientVM(3,
+            c1 -> c1.withPoolSubscription(true).withServerConnection(server1Port));
+    client4 =
+        cluster.startClientVM(4,
+            c -> c.withPoolSubscription(true).withServerConnection(server2Port));
 
     client3.invoke(() -> executeQuery());
     client4.invoke(() -> executeQuery());
@@ -252,7 +267,7 @@ public class QueryMonitorDUnitTest {
     server1.invoke(() -> populateRegion(0, 100));
 
     int server1Port = server1.getPort();
-    client3 = cluster.startClientVM(3, ccf -> {
+    client3 = cluster.startClientVM(3, new Properties(), ccf -> {
       configureClientCacheFactory(ccf, server1Port);
     });
 
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/security/ClientCQAuthDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/security/ClientCQAuthDUnitTest.java
index 1a9dbd2..9fed728 100644
--- a/geode-cq/src/distributedTest/java/org/apache/geode/security/ClientCQAuthDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/security/ClientCQAuthDUnitTest.java
@@ -14,26 +14,21 @@
  */
 package org.apache.geode.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.apache.geode.security.SecurityTestUtil.assertNotAuthorized;
-import static org.apache.geode.security.SecurityTestUtil.createClientCache;
-import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
 
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.Pool;
 import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.query.CqAttributes;
 import org.apache.geode.cache.query.CqAttributesFactory;
 import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.ClientVM;
 import org.apache.geode.test.dunit.rules.ClusterStartupRule;
 import org.apache.geode.test.junit.categories.SecurityTest;
 import org.apache.geode.test.junit.rules.ServerStarterRule;
@@ -43,34 +38,39 @@ public class ClientCQAuthDUnitTest {
 
   private static final String REGION_NAME = "AuthRegion";
 
-  private VM client1;
-  private VM client2;
-  private VM client3;
+  private ClientVM client1;
+  private ClientVM client2;
+  private ClientVM client3;
 
   @Rule
-  public ClusterStartupRule startupRule = new ClusterStartupRule();
+  public ClusterStartupRule cluster = new ClusterStartupRule();
 
   @Rule
   public ServerStarterRule server = new ServerStarterRule()
-      .withProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName())
+      .withSecurityManager(SimpleTestSecurityManager.class)
       .withRegion(RegionShortcut.REPLICATE, REGION_NAME);
 
-  @Before
-  public void setUp() {
-    client1 = startupRule.getVM(1);
-    client2 = startupRule.getVM(2);
-    client3 = startupRule.getVM(3);
-  }
-
   @Test
-  public void verifyCQPermissions() {
+  public void verifyCQPermissions() throws Exception {
     String query = "select * from /AuthRegion";
     int serverPort = server.getPort();
 
+    client1 = cluster.startClientVM(1, c2 -> c2.withCredential("test", "test")
+        .withPoolSubscription(true)
+        .withServerConnection(serverPort));
+    client2 =
+        cluster.startClientVM(2, c1 -> c1.withCredential("clusterManageQuery", "clusterManageQuery")
+            .withPoolSubscription(true)
+            .withServerConnection(serverPort));
+    client3 =
+        cluster.startClientVM(3,
+            c -> c.withCredential("clusterManageQuery,dataRead", "clusterManageQuery,dataRead")
+                .withPoolSubscription(true)
+                .withServerConnection(serverPort));
+
     // client has no permission whatsoever
     client1.invoke(() -> {
-      ClientCache cache = createClientCache("test", "test", serverPort);
-      final Region region = createProxyRegion(cache, REGION_NAME);
+      final Region region = ClusterStartupRule.clientCacheRule.createProxyRegion(REGION_NAME);
       Pool pool = PoolManager.find(region);
       QueryService qs = pool.getQueryService();
       CqAttributes cqa = new CqAttributesFactory().create();
@@ -85,8 +85,7 @@ public class ClientCQAuthDUnitTest {
 
     // client2 has part of the permission
     client2.invoke(() -> {
-      ClientCache cache = createClientCache("clusterManageQuery", "clusterManageQuery", serverPort);
-      final Region region = createProxyRegion(cache, REGION_NAME);
+      final Region region = ClusterStartupRule.clientCacheRule.createProxyRegion(REGION_NAME);
       Pool pool = PoolManager.find(region);
       QueryService qs = pool.getQueryService();
       CqAttributes cqa = new CqAttributesFactory().create();
@@ -100,9 +99,7 @@ public class ClientCQAuthDUnitTest {
 
     // client3 has all the permissions
     client3.invoke(() -> {
-      ClientCache cache = createClientCache("clusterManageQuery,dataRead",
-          "clusterManageQuery,dataRead", serverPort);
-      Region region = createProxyRegion(cache, REGION_NAME);
+      Region region = ClusterStartupRule.clientCacheRule.createProxyRegion(REGION_NAME);
       Pool pool = PoolManager.find(region);
       QueryService qs = pool.getQueryService();
       CqAttributes cqa = new CqAttributesFactory().create();
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/security/MultiUserAPIDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/security/MultiUserAPIDUnitTest.java
index 8272190..fa4bd5b 100644
--- a/geode-cq/src/distributedTest/java/org/apache/geode/security/MultiUserAPIDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/security/MultiUserAPIDUnitTest.java
@@ -14,311 +14,142 @@
  */
 package org.apache.geode.security;
 
-import static org.apache.geode.security.SecurityTestUtils.NO_EXCEPTION;
-import static org.apache.geode.test.dunit.Assert.fail;
-import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
-import java.io.IOException;
 import java.util.Properties;
 
-import javax.net.ssl.SSLException;
-import javax.net.ssl.SSLHandshakeException;
-
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionService;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.Pool;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.query.CqAttributesFactory;
-import org.apache.geode.cache.query.CqException;
 import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.Query;
-import org.apache.geode.distributed.ConfigurationProperties;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.PoolManagerImpl;
-import org.apache.geode.security.generator.CredentialGenerator;
-import org.apache.geode.security.generator.DummyCredentialGenerator;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
-@Category({SecurityTest.class})
-public class MultiUserAPIDUnitTest extends ClientAuthorizationTestCase {
-
-  private static final String[] serverIgnoredExceptions =
-      {AuthenticationRequiredException.class.getName(),
-          AuthenticationFailedException.class.getName(), GemFireSecurityException.class.getName(),
-          ClassNotFoundException.class.getName(), IOException.class.getName(),
-          SSLException.class.getName(), SSLHandshakeException.class.getName()};
-
-  private static final String[] clientIgnoredExceptions =
-      {AuthenticationRequiredException.class.getName(),
-          AuthenticationFailedException.class.getName(), SSLHandshakeException.class.getName()};
-
-  @Test
-  public void testSingleUserUnsupportedAPIs() {
-    // Start servers
-    // Start clients with multiuser-authentication set to false
-    setUpVMs(new DummyCredentialGenerator(), false);
-    client1.invoke(() -> verifyDisallowedOps(false));
+import org.apache.geode.examples.SimpleSecurityManager;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.rules.ClientCacheRule;
+
+public class MultiUserAPIDUnitTest {
+  @ClassRule
+  public static ClusterStartupRule cluster = new ClusterStartupRule();
+
+  private static MemberVM server1, server2;
+
+  @Rule
+  public ClientCacheRule client = new ClientCacheRule();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    server1 = cluster.startServerVM(0, s -> s.withSecurityManager(SimpleSecurityManager.class)
+        .withConnectionToLocator(ClusterStartupRule.getDUnitLocatorPort()));
+    server2 = cluster.startServerVM(0, s -> s.withSecurityManager(SimpleSecurityManager.class)
+        .withConnectionToLocator(ClusterStartupRule.getDUnitLocatorPort()));
+    server1.invoke(() -> {
+      ClusterStartupRule.memberStarter.createRegion(RegionShortcut.REPLICATE, "authRegion");
+    });
   }
 
   @Test
-  public void testMultiUserUnsupportedAPIs() {
-    // Start servers.
-    // Start clients with multiuser-authentication set to true.
-    setUpVMs(new DummyCredentialGenerator(), true);
-    client1.invoke(() -> verifyDisallowedOps(true));
-  }
-
-  private void verifyDisallowedOps(final boolean multiUserMode) throws Exception {
-    String op = "unknown";
-    boolean success = false;
-
-    if (!multiUserMode) {
-      success = false;
-
-      try {
-        // Attempt cache.createAuthenticatedCacheView() and expect an exception, fail otherwise
-        op = "Pool.createSecureUserCache()";
-        GemFireCacheImpl.getInstance().createAuthenticatedView(new Properties(), "testPool");
-      } catch (IllegalStateException uoe) {
-        getLogWriter().info(op + ": Got expected exception: " + uoe);
-        success = true;
-      }
-
-      if (!success) {
-        fail("Did not get exception while doing " + op);
-      }
-
-    } else { // multiuser mode
-      Region realRegion = GemFireCacheImpl.getInstance().getRegion(SecurityTestUtils.REGION_NAME);
-      Region proxyRegion =
-          SecurityTestUtils.getProxyCaches(0).getRegion(SecurityTestUtils.REGION_NAME);
-      Pool pool = PoolManagerImpl.getPMI().find("testPool");
-
-      for (int i = 0; i <= 27; i++) {
-        success = false;
-        try {
-          switch (i) {
-            // Attempt (real) Region.create/put/get/containsKeyOnServer/destroy/
-            // destroyRegion/clear/remove/registerInterest/unregisterInterest()
-            // and expect an exception, fail otherwise.
-            case 0:
-              op = "Region.create()";
-              realRegion.create("key", "value");
-              break;
-            case 1:
-              op = "Region.put()";
-              realRegion.put("key", "value");
-              break;
-            case 2:
-              op = "Region.get()";
-              realRegion.get("key");
-              break;
-            case 3:
-              op = "Region.containsKeyOnServer()";
-              realRegion.containsKeyOnServer("key");
-              break;
-            case 4:
-              op = "Region.remove()";
-              realRegion.remove("key");
-              break;
-            case 5:
-              op = "Region.destroy()";
-              realRegion.destroy("key");
-              break;
-            case 6:
-              op = "Region.destroyRegion()";
-              realRegion.destroyRegion();
-              break;
-            case 7:
-              op = "Region.registerInterest()";
-              realRegion.registerInterest("key");
-              break;
-            // case 8:
-            // op = "Region.unregisterInterest()";
-            // realRegion.unregisterInterest("key");
-            // break;
-            case 8:
-              op = "Region.clear()";
-              realRegion.clear();
-              break;
-            // Attempt ProxyRegion.createSubregion/forceRolling/
-            // getAttributesMutator/registerInterest/loadSnapShot/saveSnapshot/
-            // setUserAttribute/unregisterInterest/writeToDisk
-            // and expect an exception, fail otherwise.
-            case 9:
-              op = "ProxyRegion.createSubregion()";
-              proxyRegion.createSubregion("subregion", null);
-              break;
-            case 10:
-              op = "ProxyRegion.forceRolling()";
-              proxyRegion.forceRolling();
-              break;
-            case 11:
-              op = "ProxyRegion.getAttributesMutator()";
-              proxyRegion.getAttributesMutator();
-              break;
-            case 12:
-              op = "ProxyRegion.registerInterest()";
-              proxyRegion.registerInterest("key");
-              break;
-            case 13:
-              op = "ProxyRegion.loadSnapshot()";
-              proxyRegion.loadSnapshot(null);
-              break;
-            case 14:
-              op = "ProxyRegion.saveSnapshot()";
-              proxyRegion.saveSnapshot(null);
-              break;
-            case 15:
-              op = "ProxyRegion.setUserAttribute()";
-              proxyRegion.setUserAttribute(null);
-              break;
-            case 16:
-              op = "ProxyRegion.unregisterInterestRegex()";
-              proxyRegion.unregisterInterestRegex("*");
-              break;
-            // Attempt FunctionService.onRegion/onServer/s(pool) and expect an
-            // exception, fail otherwise.
-            case 17:
-              op = "FunctionService.onRegion()";
-              FunctionService.onRegion(realRegion);
-              break;
-            case 18:
-              op = "FunctionService.onServer(pool)";
-              FunctionService.onServer(pool);
-              break;
-            case 19:
-              op = "FunctionService.onServers(pool)";
-              FunctionService.onServers(pool);
-              break;
-            // Attempt
-            // QueryService.newQuery().execute()/newCq().execute/executeWithInitialResults()
-            case 20:
-              op = "QueryService.newQuery.execute()";
-              Query query = pool.getQueryService()
-                  .newQuery("SELECT * FROM /" + SecurityTestUtils.REGION_NAME);
-              query.execute();
-              break;
-            case 21:
-              op = "QueryService.newCq.execute()";
-              CqQuery cqQuery =
-                  pool.getQueryService().newCq("SELECT * FROM /" + SecurityTestUtils.REGION_NAME,
-                      new CqAttributesFactory().create());
-              try {
-                cqQuery.execute();
-              } catch (CqException ce) {
-                throw (Exception) ce.getCause();
-              }
-              break;
-            case 22:
-              op = "QueryService.newCq.executeWithInitialResults()";
-              cqQuery =
-                  pool.getQueryService().newCq("SELECT * FROM /" + SecurityTestUtils.REGION_NAME,
-                      new CqAttributesFactory().create());
-              try {
-                cqQuery.executeWithInitialResults();
-              } catch (CqException ce) {
-                throw (Exception) ce.getCause();
-              }
-              break;
-            // Attempt ProxyQueryService.getIndex/createIndex/removeIndex() and
-            // expect an exception, fail otherwise.
-            case 23:
-              op = "ProxyQueryService().getIndexes()";
-              SecurityTestUtils.getProxyCaches(0).getQueryService().getIndexes(null);
-              break;
-            case 24:
-              op = "ProxyQueryService().createIndex()";
-              SecurityTestUtils.getProxyCaches(0).getQueryService().createIndex(null, null, null);
-              break;
-            case 25:
-              op = "ProxyQueryService().removeIndexes()";
-              SecurityTestUtils.getProxyCaches(0).getQueryService().removeIndexes();
-              break;
-            case 26:
-              op = "ProxyRegion.localDestroy()";
-              proxyRegion.localDestroy("key");
-              break;
-            case 27:
-              op = "ProxyRegion.localInvalidate()";
-              proxyRegion.localInvalidate("key");
-              break;
-            default:
-              fail("Unknown op code: " + i);
-              break;
-          }
-
-        } catch (UnsupportedOperationException uoe) {
-          getLogWriter().info(op + ": Got expected exception: " + uoe);
-          success = true;
-        }
-        if (!success) {
-          fail("Did not get exception while doing " + op);
-        }
-      }
-    }
+  public void testSingleUserUnsupportedAPIs() throws Exception {
+    client.withCredential("stranger", "stranger").withMultiUser(false)
+        .withServerConnection(server1.getPort(), server2.getPort());
+    ClientCache clientCache = client.createCache();
+
+    assertThatThrownBy(() -> clientCache.createAuthenticatedView(new Properties()))
+        .isInstanceOf(IllegalStateException.class)
+        .hasMessageContaining("did not have multiuser-authentication set to true");
   }
 
-  private void setUpVMs(final CredentialGenerator gen, final boolean multiUser) {
-    Properties extraProps = gen.getSystemProperties();
-    Properties javaProps = gen.getJavaProperties();
-    String authenticator = gen.getAuthenticator();
-    String authInit = gen.getAuthInit();
-
-    getLogWriter().info("testValidCredentials: Using scheme: " + gen.classCode());
-    getLogWriter().info("testValidCredentials: Using authenticator: " + authenticator);
-    getLogWriter().info("testValidCredentials: Using authinit: " + authInit);
-
-    // Start the servers
-    int port1 = server1
-        .invoke(() -> createCacheServer(authenticator, extraProps, javaProps));
-    int port2 = server2
-        .invoke(() -> createCacheServer(authenticator, extraProps, javaProps));
-
-    // Start the clients with valid credentials
-    Properties credentials1 = gen.getValidCredentials(1);
-    Properties javaProps1 = gen.getJavaProperties();
-    getLogWriter().info(
-        "testValidCredentials: For first client credentials: " + credentials1 + " : " + javaProps1);
-
-    Properties credentials2 = gen.getValidCredentials(2);
-    Properties javaProps2 = gen.getJavaProperties();
-    getLogWriter().info("testValidCredentials: For second client credentials: " + credentials2
-        + " : " + javaProps2);
-
-    client1.invoke(() -> createCacheClient(authInit, credentials1, javaProps1, port1, port2, 0,
-        multiUser, NO_EXCEPTION));
-  }
-
-  private int createCacheServer(final String authenticator, final Properties extraProps,
-      final Properties javaProps) {
-    Properties authProps = new Properties();
-    if (extraProps != null) {
-      authProps.putAll(extraProps);
-    }
-
-    if (authenticator != null) {
-      authProps.setProperty(ConfigurationProperties.SECURITY_CLIENT_AUTHENTICATOR, authenticator);
-    }
-
-    return SecurityTestUtils.createCacheServer(authProps, javaProps, 0, NO_EXCEPTION);
-  }
+  @Test
+  public void testMultiUserUnsupportedAPIs() throws Exception {
+    client.withCredential("stranger", "stranger")
+        .withPoolSubscription(true)
+        .withMultiUser(true)
+        .withServerConnection(server1.getPort(), server2.getPort());
+    client.createCache();
+    Region realRegion = client.createProxyRegion("authRegion");
+    Pool pool = client.getCache().getDefaultPool();
+
+    RegionService proxyCache = client.createAuthenticatedView("data", "data");
+    Region proxyRegion = proxyCache.getRegion("authRegion");
+
+    assertThatThrownBy(() -> realRegion.create("key", "value"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.put("key", "value"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.get("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.containsKeyOnServer("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.remove("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.destroy("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.destroyRegion())
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.registerInterest("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> realRegion.clear()).isInstanceOf(UnsupportedOperationException.class);
+
+
+    assertThatThrownBy(() -> proxyRegion.createSubregion("subRegion", null))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.forceRolling())
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.getAttributesMutator())
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.loadSnapshot(null))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.saveSnapshot(null))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.registerInterest("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.setUserAttribute(null))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.unregisterInterestRegex("*"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.localDestroy("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyRegion.localInvalidate("key"))
+        .isInstanceOf(UnsupportedOperationException.class);
+
+    assertThatThrownBy(() -> FunctionService.onRegion(realRegion))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> FunctionService.onServer(pool))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> FunctionService.onServers(pool))
+        .isInstanceOf(UnsupportedOperationException.class);
+
+
+    assertThatThrownBy(() -> {
+      Query query = pool.getQueryService().newQuery("SELECT * FROM /authRegion");
+      query.execute();
+    }).isInstanceOf(UnsupportedOperationException.class);
+    CqQuery cqQuery =
+        pool.getQueryService().newCq("SELECT * FROM /authRegion",
+            new CqAttributesFactory().create());
+    assertThatThrownBy(() -> cqQuery.execute())
+        .hasCauseInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> cqQuery.executeWithInitialResults())
+        .hasCauseInstanceOf(UnsupportedOperationException.class);
+
+    assertThatThrownBy(() -> proxyCache.getQueryService().getIndexes())
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyCache.getQueryService().getIndexes(null))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyCache.getQueryService().createIndex(null, null, null))
+        .isInstanceOf(UnsupportedOperationException.class);
+    assertThatThrownBy(() -> proxyCache.getQueryService().removeIndexes())
+        .isInstanceOf(UnsupportedOperationException.class);
 
-  // a
-  protected static void createCacheClient(final String authInit, final Properties authProps,
-      final Properties javaProps, final int[] ports, final int numConnections,
-      final boolean multiUserMode, final int expectedResult) {
-    SecurityTestUtils.createCacheClient(authInit, authProps, javaProps, ports, numConnections,
-        multiUserMode, expectedResult); // invokes SecurityTestUtils 2
-  }
 
-  // b
-  private void createCacheClient(final String authInit, final Properties authProps,
-      final Properties javaProps, final int port1, final int port2, final int numConnections,
-      final boolean multiUserMode, final int expectedResult) {
-    createCacheClient(authInit, authProps, javaProps, new int[] {port1, port2}, numConnections,
-        multiUserMode, expectedResult); // invokes a
   }
 }
diff --git a/geode-dunit/src/distributedTest/java/org/apache/geode/test/dunit/rules/tests/ClusterStartupRuleCanSpecifyOlderVersionsDUnitTest.java b/geode-dunit/src/distributedTest/java/org/apache/geode/test/dunit/rules/tests/ClusterStartupRuleCanSpecifyOlderVersionsDUnitTest.java
index 42a108b..84b988e 100644
--- a/geode-dunit/src/distributedTest/java/org/apache/geode/test/dunit/rules/tests/ClusterStartupRuleCanSpecifyOlderVersionsDUnitTest.java
+++ b/geode-dunit/src/distributedTest/java/org/apache/geode/test/dunit/rules/tests/ClusterStartupRuleCanSpecifyOlderVersionsDUnitTest.java
@@ -77,8 +77,8 @@ public class ClusterStartupRuleCanSpecifyOlderVersionsDUnitTest {
 
   @Test
   public void clientVersioningTest() throws Exception {
-    ClientVM locator = csRule.startClientVM(0, new Properties(), (cf) -> {
-    }, version);
+    ClientVM locator = csRule.startClientVM(0, version, new Properties(), (cf) -> {
+    });
     String locatorVMVersion = locator.getVM().getVersion();
     String locatorActualVersion = locator.invoke(GemFireVersion::getGemFireVersion);
     assertThat(locatorVMVersion).isEqualTo(version);
diff --git a/geode-dunit/src/main/java/org/apache/geode/security/SecurityTestUtil.java b/geode-dunit/src/main/java/org/apache/geode/security/SecurityTestUtil.java
index e6561b0..ff0376e 100644
--- a/geode-dunit/src/main/java/org/apache/geode/security/SecurityTestUtil.java
+++ b/geode-dunit/src/main/java/org/apache/geode/security/SecurityTestUtil.java
@@ -32,13 +32,18 @@ import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.internal.Version;
 import org.apache.geode.security.templates.UserPasswordAuthInit;
 
+/**
+ * @deprecated use @org.apache.geode.test.junit.rules.ClientCacheRule instead
+ */
 public class SecurityTestUtil {
 
+  @Deprecated
   public static ClientCache createClientCache(String username, String password, int serverPort) {
     Properties props = new Properties();
     return createClientCache(username, password, serverPort, props);
   }
 
+  @Deprecated
   public static ClientCache createClientCache(String username, String password, int serverPort,
       Properties extraProperties) {
     Properties props = new Properties();
@@ -56,10 +61,12 @@ public class SecurityTestUtil {
     return cache;
   }
 
+  @Deprecated
   public static Region createProxyRegion(ClientCache cache, String regionName) {
     return cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(regionName);
   }
 
+  @Deprecated
   public static void assertNotAuthorized(ThrowableAssert.ThrowingCallable shouldRaiseThrowable,
       String permString) {
     assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClusterStartupRule.java b/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClusterStartupRule.java
index c336165..0687cc6 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClusterStartupRule.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/ClusterStartupRule.java
@@ -16,7 +16,6 @@
 package org.apache.geode.test.dunit.rules;
 
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.dunit.Host.getHost;
 import static org.apache.geode.test.dunit.standalone.DUnitLauncher.NUM_VMS;
@@ -40,7 +39,6 @@ import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.security.templates.UserPasswordAuthInit;
 import org.apache.geode.test.dunit.DUnitEnv;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.RMIException;
@@ -72,7 +70,7 @@ public class ClusterStartupRule extends ExternalResource implements Serializable
   /**
    * This is only available in each Locator/Server VM, not in the controller (test) VM.
    */
-  public static MemberStarterRule memberStarter;
+  public static MemberStarterRule<?> memberStarter;
   public static ClientCacheRule clientCacheRule;
 
   public static InternalCache getCache() {
@@ -252,15 +250,14 @@ public class ClusterStartupRule extends ExternalResource implements Serializable
     return memberVM;
   }
 
-  public ClientVM startClientVM(int index, Properties properties,
-      SerializableConsumerIF<ClientCacheFactory> cacheFactorySetup, String clientVersion)
-      throws Exception {
+  public ClientVM startClientVM(int index, String clientVersion,
+      SerializableConsumerIF<ClientCacheRule> clientCacheRuleSetUp) throws Exception {
     VM client = getVM(index, clientVersion);
     Exception error = client.invoke(() -> {
-      clientCacheRule =
-          new ClientCacheRule().withProperties(properties).withCacheSetup(cacheFactorySetup);
+      clientCacheRule = new ClientCacheRule();
       try {
-        clientCacheRule.before();
+        clientCacheRuleSetUp.accept(clientCacheRule);
+        clientCacheRule.createCache();
         return null;
       } catch (Exception e) {
         return e;
@@ -275,41 +272,22 @@ public class ClusterStartupRule extends ExternalResource implements Serializable
   }
 
   public ClientVM startClientVM(int index,
-      SerializableConsumerIF<ClientCacheFactory> cacheFactorySetup) throws Exception {
-    return startClientVM(index, new Properties(), cacheFactorySetup,
-        VersionManager.CURRENT_VERSION);
+      SerializableConsumerIF<ClientCacheRule> clientCacheRuleSetUp) throws Exception {
+    return startClientVM(index, VersionManager.CURRENT_VERSION, clientCacheRuleSetUp);
   }
 
-  public ClientVM startClientVM(int index, Properties properties,
-      SerializableConsumerIF<ClientCacheFactory> cacheFactorySetup) throws Exception {
-    return startClientVM(index, properties, cacheFactorySetup, VersionManager.CURRENT_VERSION);
+  public ClientVM startClientVM(int index, String clientVersion, Properties properties,
+      SerializableConsumerIF<ClientCacheFactory> cacheFactorySetup)
+      throws Exception {
+    return startClientVM(index, clientVersion,
+        c -> c.withProperties(properties).withCacheSetup(cacheFactorySetup));
   }
 
-  // convenient startClientMethod
-  public ClientVM startClientVM(int index, String username, String password,
-      boolean subscriptionEnabled, int... serverPorts) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(UserPasswordAuthInit.USER_NAME, username);
-    props.setProperty(UserPasswordAuthInit.PASSWORD, password);
-    props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
 
-    return startClientVM(index, props, (ccf) -> {
-      ccf.setPoolSubscriptionEnabled(subscriptionEnabled);
-      for (int serverPort : serverPorts) {
-        ccf.addPoolServer("localhost", serverPort);
-      }
-    });
-  }
-
-  // convenient startClientMethod
-  public ClientVM startClientVM(int index, boolean subscriptionEnabled, int... serverPorts)
-      throws Exception {
-    return startClientVM(index, ccf -> {
-      ccf.setPoolSubscriptionEnabled(subscriptionEnabled);
-      for (int port : serverPorts) {
-        ccf.addPoolServer("localhost", port);
-      }
-    });
+  public ClientVM startClientVM(int index, Properties properties,
+      SerializableConsumerIF<ClientCacheFactory> cacheFactorySetup) throws Exception {
+    return startClientVM(index,
+        c -> c.withProperties(properties).withCacheSetup(cacheFactorySetup));
   }
 
   /**
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/MemberVM.java b/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/MemberVM.java
index 2a99cb7..b7c4a34 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/MemberVM.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/dunit/rules/MemberVM.java
@@ -32,7 +32,7 @@ import org.apache.geode.test.junit.rules.Server;
 import org.apache.geode.test.junit.rules.VMProvider;
 
 public class MemberVM extends VMProvider implements Member {
-  private Logger logger = LogService.getLogger();
+  private static Logger logger = LogService.getLogger();
   protected Member member;
   protected VM vm;
 
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ClientCacheRule.java b/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ClientCacheRule.java
index 28c8544..605de66 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ClientCacheRule.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ClientCacheRule.java
@@ -15,25 +15,36 @@
 
 package org.apache.geode.test.junit.rules;
 
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Properties;
 import java.util.function.Consumer;
 
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionService;
 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.internal.net.SocketCreatorFactory;
+import org.apache.geode.security.templates.UserPasswordAuthInit;
 import org.apache.geode.test.junit.rules.serializable.SerializableExternalResource;
 
 public class ClientCacheRule extends SerializableExternalResource {
   private ClientCache cache;
   private ClientCacheFactory cacheFactory;
-  private Consumer<ClientCacheFactory> cacheSetup;
+  private List<Consumer<ClientCacheFactory>> cacheSetups;
   private Properties properties;
+  private boolean autoCreate;
 
   public ClientCacheRule() {
     properties = new Properties();
     properties.setProperty(MCAST_PORT, "0");
+    properties.setProperty(LOCATORS, "");
+    cacheSetups = new ArrayList<>();
   }
 
   public ClientCacheRule withProperties(Properties properties) {
@@ -41,21 +52,80 @@ public class ClientCacheRule extends SerializableExternalResource {
     return this;
   }
 
+  public ClientCacheRule withCredential(String username, String password) {
+    properties.setProperty(UserPasswordAuthInit.USER_NAME, username);
+    properties.setProperty(UserPasswordAuthInit.PASSWORD, password);
+    properties.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
+    return this;
+  }
+
   public ClientCacheRule withProperty(String key, String value) {
     properties.put(key, value);
     return this;
   }
 
   public ClientCacheRule withCacheSetup(Consumer<ClientCacheFactory> setup) {
-    cacheSetup = setup;
+    cacheSetups.add(setup);
     return this;
   }
 
-  @Override
-  public void before() throws Exception {
+  public ClientCacheRule withPoolSubscription(boolean enabled) {
+    withCacheSetup(cf -> cf.setPoolSubscriptionEnabled(enabled));
+    return this;
+  }
+
+  public ClientCacheRule withServerConnection(int... serverPorts) {
+    withCacheSetup(cf -> {
+      for (int serverPort : serverPorts) {
+        cf.addPoolServer("localhost", serverPort);
+      }
+    });
+    return this;
+  }
+
+  public ClientCacheRule withLocatorConnection(int... serverPorts) {
+    withCacheSetup(cf -> {
+      for (int serverPort : serverPorts) {
+        cf.addPoolLocator("localhost", serverPort);
+      }
+    });
+    return this;
+  }
+
+  public ClientCacheRule withMultiUser(boolean enabled) {
+    withCacheSetup(cf -> cf.setPoolMultiuserAuthentication(enabled));
+    return this;
+  }
+
+  public ClientCacheRule withAutoCreate() {
+    this.autoCreate = true;
+    return this;
+  }
+
+  public ClientCache createCache() throws Exception {
     cacheFactory = new ClientCacheFactory(properties);
-    cacheSetup.accept(cacheFactory);
+    cacheSetups.stream().forEach(setup -> setup.accept(cacheFactory));
     cache = cacheFactory.create();
+    return cache;
+  }
+
+  public Region createProxyRegion(String regionPath) {
+    return cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(regionPath);
+  }
+
+  public RegionService createAuthenticatedView(String username, String password) {
+    Properties properties = new Properties();
+    properties.setProperty(UserPasswordAuthInit.USER_NAME, username);
+    properties.setProperty(UserPasswordAuthInit.PASSWORD, password);
+    properties.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
+    return cache.createAuthenticatedView(properties);
+  }
+
+  @Override
+  public void before() throws Exception {
+    if (autoCreate) {
+      createCache();
+    }
   }
 
   @Override
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/MemberStarterRule.java b/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/MemberStarterRule.java
index 0b4bd97..2dbf988 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/MemberStarterRule.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/MemberStarterRule.java
@@ -52,6 +52,10 @@ import org.apache.commons.lang3.ArrayUtils;
 import org.assertj.core.api.Assertions;
 import org.awaitility.core.ConditionTimeoutException;
 
+import org.apache.geode.cache.PartitionAttributesFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionFactory;
+import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
@@ -339,6 +343,41 @@ public abstract class MemberStarterRule<T> extends SerializableExternalResource
   /**
    * Invoked in serverVM
    */
+
+  /**
+   * convenience method to create a region with customized regionFactory
+   *
+   * @param regionFactoryConsumer a lamda that allows you to customize the regionFactory
+   */
+  public Region createRegion(RegionShortcut type, String name,
+      Consumer<RegionFactory> regionFactoryConsumer) {
+    RegionFactory regionFactory = getCache().createRegionFactory(type);
+    regionFactoryConsumer.accept(regionFactory);
+    return regionFactory.create(name);
+  }
+
+  public Region createRegion(RegionShortcut type, String name) {
+    return getCache().createRegionFactory(type).create(name);
+  }
+
+  /**
+   * convenience method to create a partition region with customized regionFactory and a customized
+   * PartitionAttributeFactory
+   *
+   * @param regionFactoryConsumer a lamda that allows you to customize the regionFactory
+   * @param attributesFactoryConsumer a lamda that allows you to customize the
+   *        partitionAttributeFactory
+   */
+  public Region createPartitionRegion(String name, Consumer<RegionFactory> regionFactoryConsumer,
+      Consumer<PartitionAttributesFactory> attributesFactoryConsumer) {
+    return createRegion(RegionShortcut.PARTITION, name, rf -> {
+      regionFactoryConsumer.accept(rf);
+      PartitionAttributesFactory attributeFactory = new PartitionAttributesFactory();
+      attributesFactoryConsumer.accept(attributeFactory);
+      rf.setPartitionAttributes(attributeFactory.create());
+    });
+  }
+
   public void waitTillCacheClientProxyHasBeenPaused() {
     await().until(() -> {
       CacheClientNotifier clientNotifier = CacheClientNotifier.getInstance();
diff --git a/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ServerStarterRule.java b/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ServerStarterRule.java
index b95e6aa..ae7c12b 100644
--- a/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ServerStarterRule.java
+++ b/geode-dunit/src/main/java/org/apache/geode/test/junit/rules/ServerStarterRule.java
@@ -22,11 +22,8 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
-import java.util.function.Consumer;
 
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.PartitionAttributesFactory;
-import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.server.CacheServer;
@@ -163,36 +160,6 @@ public class ServerStarterRule extends MemberStarterRule<ServerStarterRule> impl
     return this;
   }
 
-  /**
-   * convenience method to create a region with customized regionFactory
-   *
-   * @param regionFactoryConsumer a lamda that allows you to customize the regionFactory
-   */
-  public Region createRegion(RegionShortcut type, String name,
-      Consumer<RegionFactory> regionFactoryConsumer) {
-    RegionFactory regionFactory = getCache().createRegionFactory(type);
-    regionFactoryConsumer.accept(regionFactory);
-    return regionFactory.create(name);
-  }
-
-  /**
-   * convenience method to create a partition region with customized regionFactory and a customized
-   * PartitionAttributeFactory
-   *
-   * @param regionFactoryConsumer a lamda that allows you to customize the regionFactory
-   * @param attributesFactoryConsumer a lamda that allows you to customize the
-   *        partitionAttributeFactory
-   */
-  public Region createPartitionRegion(String name, Consumer<RegionFactory> regionFactoryConsumer,
-      Consumer<PartitionAttributesFactory> attributesFactoryConsumer) {
-    return createRegion(RegionShortcut.PARTITION, name, rf -> {
-      regionFactoryConsumer.accept(rf);
-      PartitionAttributesFactory attributeFactory = new PartitionAttributesFactory();
-      attributesFactoryConsumer.accept(attributeFactory);
-      rf.setPartitionAttributes(attributeFactory.create());
-    });
-  }
-
   public void startServer(Properties properties, int locatorPort) {
     withProperties(properties).withConnectionToLocator(locatorPort).startServer();
   }