You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kh...@apache.org on 2017/03/28 16:46:52 UTC

[24/35] geode git commit: GEODE-2395: use random ports when starting the jmx manager and http services

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/PostProcessorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/PostProcessorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/PostProcessorDUnitTest.java
index ed16f0c..a650a80 100644
--- a/geode-core/src/test/java/org/apache/geode/security/PostProcessorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/PostProcessorDUnitTest.java
@@ -14,37 +14,60 @@
  */
 package org.apache.geode.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import org.apache.geode.cache.EntryEvent;
 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.ClientCacheFactory;
 import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
 import org.apache.geode.cache.client.Pool;
 import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.query.SelectResults;
 import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-@Category({DistributedTest.class, SecurityTest.class})
-public class PostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
-  public Properties getProperties() {
-    Properties properties = super.getProperties();
-    properties.setProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName());
-    return properties;
+@Category({DistributedTest.class, SecurityTest.class})
+public class PostProcessorDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .withProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName()).startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
   }
 
   @Test
@@ -54,8 +77,8 @@ public class PostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
     keys.add("key2");
 
     client1.invoke(() -> {
-      ClientCache cache = createClientCache("super-user", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
 
       // post process for get
       Object value = region.get("key3");
@@ -72,8 +95,8 @@ public class PostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
   @Test
   public void testPostProcessQuery() {
     client1.invoke(() -> {
-      ClientCache cache = createClientCache("super-user", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
 
       // post process for query
       String query = "select * from /AuthRegion";
@@ -99,8 +122,7 @@ public class PostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
   @Test
   public void testRegisterInterestPostProcess() {
     client1.invoke(() -> {
-      ClientCache cache = new ClientCacheFactory(createClientProperties("super-user", "1234567"))
-          .setPoolSubscriptionEnabled(true).addPoolServer("localhost", serverPort).create();
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
 
       ClientRegionFactory factory = cache.createClientRegionFactory(ClientRegionShortcut.PROXY);
       factory.addCacheListener(new CacheListenerAdapter() {
@@ -117,8 +139,8 @@ public class PostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
     });
 
     client2.invoke(() -> {
-      ClientCache cache = createClientCache("dataUser", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("dataUser", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
       region.put("key1", "value2");
     });
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/SecurityTestUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityTestUtil.java b/geode-core/src/test/java/org/apache/geode/security/SecurityTestUtil.java
new file mode 100644
index 0000000..5d5c214
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityTestUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.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.assertj.core.api.Assertions.assertThatThrownBy;
+
+import org.apache.geode.cache.Region;
+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.security.templates.UserPasswordAuthInit;
+import org.assertj.core.api.ThrowableAssert;
+
+import java.util.Properties;
+
+public class SecurityTestUtil {
+
+  public static ClientCache createClientCache(String username, String password, int serverPort) {
+    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");
+    ClientCache cache = new ClientCacheFactory(props).setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort).create();
+    return cache;
+  }
+
+  public static Region createProxyRegion(ClientCache cache, String regionName) {
+    return cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(regionName);
+  }
+
+  public static void assertNotAuthorized(ThrowableAssert.ThrowingCallable shouldRaiseThrowable,
+      String permString) {
+    assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
index 52a4ce4..c5d5e1d 100644
--- a/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/security/SecurityWithoutClusterConfigDUnitTest.java
@@ -24,7 +24,9 @@ import static org.junit.Assert.assertFalse;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.rules.Locator;
 import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
 import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
@@ -45,6 +47,8 @@ public class SecurityWithoutClusterConfigDUnitTest {
   @Rule
   public ServerStarterRule serverStarter = new ServerStarterRule();
 
+  private MemberVM<Locator> locator;
+
   @Before
   public void before() throws Exception {
     IgnoredException
@@ -55,7 +59,7 @@ public class SecurityWithoutClusterConfigDUnitTest {
     props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
     props.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
     props.setProperty(ENABLE_CLUSTER_CONFIGURATION, "false");
-    lsRule.startLocatorVM(0, props);
+    locator = lsRule.startLocatorVM(0, props);
   }
 
   @Test
@@ -72,7 +76,7 @@ public class SecurityWithoutClusterConfigDUnitTest {
     props.setProperty("use-cluster-configuration", "true");
 
     // initial security properties should only contain initial set of values
-    serverStarter.startServer(props, lsRule.getMember(0).getPort());
+    serverStarter.withProperties(props).withConnectionToLocator(locator.getPort()).startServer();
     DistributedSystem ds = serverStarter.getCache().getDistributedSystem();
     assertEquals(3, ds.getSecurityProperties().size());
 

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
index f367458..d061859 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/GfshShellConnectionRule.java
@@ -58,18 +58,24 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
   private boolean connected = false;
   private TemporaryFolder temporaryFolder = new TemporaryFolder();
 
-  public GfshShellConnectionRule() {}
+  public GfshShellConnectionRule() {
+    try {
+      temporaryFolder.create();
+      this.gfsh = new HeadlessGfsh(getClass().getName(), 30,
+          temporaryFolder.newFolder("gfsh_files").getAbsolutePath());
+    } catch (Exception e) {
+      throw new RuntimeException(e.getMessage(), e);
+    }
+  }
 
   public GfshShellConnectionRule(int port, PortType portType) {
+    this();
     this.portType = portType;
     this.port = port;
   }
 
   @Override
   protected void before(Description description) throws Throwable {
-    temporaryFolder.create();
-    this.gfsh = new HeadlessGfsh(getClass().getName(), 30,
-        temporaryFolder.newFolder("gfsh_files").getAbsolutePath());
     // do not auto connect if no port initialized
     if (port < 0) {
       return;
@@ -100,6 +106,19 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
     assertThat(this.connected).isTrue();
   }
 
+  public void secureConnect(int port, PortType type, String username, String password)
+      throws Exception {
+    connect(port, type, CliStrings.CONNECT__USERNAME, username, CliStrings.CONNECT__PASSWORD,
+        password);
+  }
+
+  public void secureConnectAndVerify(int port, PortType type, String username, String password)
+      throws Exception {
+    connect(port, type, CliStrings.CONNECT__USERNAME, username, CliStrings.CONNECT__PASSWORD,
+        password);
+    assertThat(this.connected).isTrue();
+  }
+
   public void connect(int port, PortType type, String... options) throws Exception {
     CliUtil.isGfshVM = true;
     final CommandStringBuilder connectCommand = new CommandStringBuilder(CliStrings.CONNECT);
@@ -142,10 +161,6 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
 
   @Override
   protected void after(Description description) throws Throwable {
-    temporaryFolder.delete();
-    if (connected) {
-      disconnect();
-    }
     close();
   }
 
@@ -156,6 +171,10 @@ public class GfshShellConnectionRule extends DescribedExternalResource {
   }
 
   public void close() throws Exception {
+    temporaryFolder.delete();
+    if (connected) {
+      disconnect();
+    }
     gfsh.executeCommand("exit");
     gfsh.terminate();
     gfsh = null;

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
index 01e346a..2d648bd 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorServerStartupRule.java
@@ -20,6 +20,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_P
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
 import static org.apache.geode.test.dunit.Host.getHost;
 
+import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.standalone.DUnitLauncher;
@@ -77,7 +78,7 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
     temporaryFolder.delete();
   }
 
-  public MemberVM startLocatorVM(int index) throws IOException {
+  public MemberVM startLocatorVM(int index) throws Exception {
     return startLocatorVM(index, new Properties());
   }
 
@@ -87,60 +88,55 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
    *
    * @return VM locator vm
    */
-  public MemberVM<Locator> startLocatorVM(int index, Properties locatorProperties)
-      throws IOException {
+  public MemberVM<Locator> startLocatorVM(int index, Properties properties) throws Exception {
     String name = "locator-" + index;
-    locatorProperties.setProperty(NAME, name);
+    properties.setProperty(NAME, name);
     File workingDir = createWorkingDirForMember(name);
     VM locatorVM = getHost(0).getVM(index);
     Locator locator = locatorVM.invoke(() -> {
       locatorStarter = new LocatorStarterRule(workingDir);
-      locatorStarter.before();
-      return locatorStarter.startLocator(locatorProperties);
+      return locatorStarter.withProperties(properties).startLocator();
     });
     members[index] = new MemberVM(locator, locatorVM);
     return members[index];
   }
 
+
   public MemberVM startServerVM(int index) throws IOException {
     return startServerVM(index, new Properties(), -1);
   }
 
-  /**
-   * starts a cache server that does not connect to a locator
-   * 
-   * @return VM node vm
-   */
-  public MemberVM startServerVM(int index, Properties properties) throws IOException {
-    return startServerVM(index, properties, -1);
-  }
-
   public MemberVM startServerVM(int index, int locatorPort) throws IOException {
     return startServerVM(index, new Properties(), locatorPort);
   }
 
-  public MemberVM startServerAsJmxManager(int index, int jmxManagerPort) throws IOException {
-    Properties properties = new Properties();
-    properties.setProperty(JMX_MANAGER_PORT, jmxManagerPort + "");
-    return startServerVM(index, properties);
+  public MemberVM startServerVM(int index, Properties properties) throws IOException {
+    return startServerVM(index, properties, -1);
   }
 
-  public MemberVM startServerAsEmbededLocator(int index, int locatorPort, int jmxManagerPort)
-      throws IOException {
+  public MemberVM startServerAsJmxManager(int index) throws IOException {
     Properties properties = new Properties();
-    properties.setProperty("start-locator", "localhost[" + locatorPort + "]");
-    if (jmxManagerPort > 0) {
-      properties.setProperty(JMX_MANAGER_PORT, jmxManagerPort + "");
-    }
-    return startServerVM(index, properties);
+    properties.setProperty(JMX_MANAGER_PORT, AvailablePortHelper.getRandomAvailableTCPPort() + "");
+    return startServerVM(index, properties, -1);
+  }
+
+  public MemberVM startServerAsEmbededLocator(int index) throws IOException {
+    String name = "server-" + index;
+    File workingDir = createWorkingDirForMember(name);
+    VM serverVM = getHost(0).getVM(index);
+    Server server = serverVM.invoke(() -> {
+      serverStarter = new ServerStarterRule(workingDir);
+      return serverStarter.withEmbeddedLocator().withName(name).withJMXManager().startServer();
+    });
+    members[index] = new MemberVM(server, serverVM);
+    return members[index];
   }
 
   /**
-   * Starts a cache server that connect to the locator running at the given port.
+   * Starts a cache server with given properties
    */
   public MemberVM startServerVM(int index, Properties properties, int locatorPort)
       throws IOException {
-
     String name = "server-" + index;
     properties.setProperty(NAME, name);
 
@@ -148,8 +144,8 @@ public class LocatorServerStartupRule extends ExternalResource implements Serial
     VM serverVM = getHost(0).getVM(index);
     Server server = serverVM.invoke(() -> {
       serverStarter = new ServerStarterRule(workingDir);
-      serverStarter.before();
-      return serverStarter.startServer(properties, locatorPort);
+      return serverStarter.withProperties(properties).withConnectionToLocator(locatorPort)
+          .startServer();
     });
     members[index] = new MemberVM(server, serverVM);
     return members[index];

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
index 5d6a0be..29f60d7 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/LocatorStarterRule.java
@@ -15,21 +15,15 @@
 
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
-import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.NAME;
 import static org.apache.geode.distributed.Locator.startLocatorAndDS;
 import static org.junit.Assert.assertTrue;
 
+import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.awaitility.Awaitility;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -44,14 +38,14 @@ import java.util.concurrent.TimeUnit;
  * use {@link LocatorServerStartupRule}.
  */
 
-public class LocatorStarterRule extends MemberStarterRule implements Locator {
+public class LocatorStarterRule extends MemberStarterRule<LocatorStarterRule> implements Locator {
 
   private transient InternalLocator locator;
 
   public LocatorStarterRule() {}
 
   public LocatorStarterRule(File workingDir) {
-    this.workingDir = workingDir.getAbsoluteFile();
+    super(workingDir);
   }
 
   public InternalLocator getLocator() {
@@ -66,44 +60,26 @@ public class LocatorStarterRule extends MemberStarterRule implements Locator {
   }
 
   public LocatorStarterRule startLocator() {
-    return startLocator(new Properties());
-  }
-
-  public LocatorStarterRule startLocator(Properties properties) {
-    if (properties == null)
-      properties = new Properties();
-    if (!properties.containsKey(NAME)) {
-      properties.setProperty(NAME, "locator");
-    }
-
-    name = properties.getProperty(NAME);
-    if (!properties.containsKey(LOG_FILE)) {
-      properties.setProperty(LOG_FILE, new File(name + ".log").getAbsolutePath());
+    normalizeProperties();
+    // start locator will start a jmx manager by default, if withJmxManager is not called explicitly
+    // the tests will use random ports by default.
+    if (jmxPort < 0) {
+      withJMXManager();
     }
 
-    if (!properties.containsKey(MCAST_PORT)) {
-      properties.setProperty(MCAST_PORT, "0");
-    }
-    if (properties.containsKey(JMX_MANAGER_PORT)) {
-      jmxPort = Integer.parseInt(properties.getProperty(JMX_MANAGER_PORT));
-      if (jmxPort > 0) {
-        if (!properties.containsKey(JMX_MANAGER)) {
-          properties.put(JMX_MANAGER, "true");
-        }
-        if (!properties.containsKey(JMX_MANAGER_START)) {
-          properties.put(JMX_MANAGER_START, "true");
-        }
-      }
-    }
     try {
+      // this will start a jmx manager and admin rest service by default
       locator = (InternalLocator) startLocatorAndDS(0, null, properties);
     } catch (IOException e) {
       throw new RuntimeException("unable to start up locator.", e);
     }
     memberPort = locator.getPort();
+    DistributionConfig config = locator.getConfig();
+    jmxPort = config.getJmxManagerPort();
+    httpPort = config.getHttpServicePort();
     locator.resetInternalLocatorFileNamesWithCorrectPortNumber(memberPort);
 
-    if (locator.getConfig().getEnableClusterConfiguration()) {
+    if (config.getEnableClusterConfiguration()) {
       Awaitility.await().atMost(65, TimeUnit.SECONDS)
           .until(() -> assertTrue(locator.isSharedConfigurationRunning()));
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
index 6165c84..3967780 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Member.java
@@ -25,5 +25,7 @@ public interface Member extends Serializable {
 
   int getJmxPort();
 
+  int getHttpPort();
+
   String getName();
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
index 7591616..3fda85a 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
@@ -16,40 +16,66 @@
 
 package org.apache.geode.test.dunit.rules;
 
-import org.apache.commons.io.FileUtils;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
+import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+
+import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.test.dunit.VM;
 import org.junit.rules.ExternalResource;
 import org.junit.rules.TemporaryFolder;
 
 import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
 
 /**
  * A server or locator inside a DUnit {@link VM}.
  */
-public abstract class MemberStarterRule extends ExternalResource implements Member {
-  protected TemporaryFolder temporaryFolder;
+public abstract class MemberStarterRule<T> extends ExternalResource implements Member {
+  protected transient TemporaryFolder temporaryFolder;
   protected String oldUserDir;
 
   protected File workingDir;
   protected int memberPort = -1;
   protected int jmxPort = -1;
+  protected int httpPort = -1;
+
   protected String name;
+  protected Properties properties = new Properties();
 
-  @Override
-  public void before() throws Exception {
+  public MemberStarterRule() {
+    this(null);
+  }
+
+  public MemberStarterRule(File workDir) {
+    workingDir = workDir;
     oldUserDir = System.getProperty("user.dir");
     if (workingDir == null) {
       temporaryFolder = new TemporaryFolder();
-      temporaryFolder.create();
-      workingDir = temporaryFolder.newFolder("locator").getAbsoluteFile();
+      try {
+        temporaryFolder.create();
+      } catch (IOException e) {
+        throw new RuntimeException(e.getMessage(), e);
+      }
+      workingDir = temporaryFolder.getRoot().getAbsoluteFile();
     }
+
     System.setProperty("user.dir", workingDir.toString());
+    // initial values
+    properties.setProperty(MCAST_PORT, "0");
+    properties.setProperty(LOCATORS, "");
   }
 
   @Override
   public void after() {
     stopMember();
-    FileUtils.deleteQuietly(workingDir);
     if (oldUserDir == null) {
       System.clearProperty("user.dir");
     } else {
@@ -60,6 +86,83 @@ public abstract class MemberStarterRule extends ExternalResource implements Memb
     }
   }
 
+  public T withProperty(String key, String value) {
+    properties.setProperty(key, value);
+    return (T) this;
+  }
+
+  public T withProperties(Properties props) {
+    if (props != null) {
+      this.properties.putAll(props);
+    }
+    return (T) this;
+  }
+
+  public T withName(String name) {
+    this.name = name;
+    properties.setProperty(NAME, name);
+    // if log-file is not already set
+    properties.putIfAbsent(LOG_FILE, new File(name + ".log").getAbsolutePath().toString());
+    return (T) this;
+  }
+
+  public T withConnectionToLocator(int locatorPort) {
+    if (locatorPort > 0) {
+      properties.setProperty(LOCATORS, "localhost[" + locatorPort + "]");
+    }
+    return (T) this;
+  }
+
+  /**
+   * be able to start JMX manager and admin rest on default ports
+   */
+  public T withJMXManager(boolean useDefault) {
+    // the real port numbers will be set after we started the server/locator.
+    this.jmxPort = 0;
+    this.httpPort = 0;
+    if (!useDefault) {
+      // do no override these properties if already exists
+      properties.putIfAbsent(JMX_MANAGER_PORT,
+          AvailablePortHelper.getRandomAvailableTCPPort() + "");
+      properties.putIfAbsent(HTTP_SERVICE_PORT,
+          AvailablePortHelper.getRandomAvailableTCPPort() + "");
+    }
+    properties.putIfAbsent(JMX_MANAGER, "true");
+    properties.putIfAbsent(JMX_MANAGER_START, "true");
+    properties.putIfAbsent(HTTP_SERVICE_BIND_ADDRESS, "localhost");
+    return (T) this;
+  }
+
+  /**
+   * start the jmx manager and admin rest on a random ports
+   */
+  public T withJMXManager() {
+    return withJMXManager(false);
+  }
+
+  protected void normalizeProperties() {
+    // if name is set via property, not with API
+    if (name == null) {
+      if (properties.containsKey(NAME)) {
+        name = properties.getProperty(NAME);
+      } else {
+        if (this instanceof ServerStarterRule)
+          name = "server";
+        else {
+          name = "locator";
+        }
+      }
+      withName(name);
+    }
+
+    // if jmxPort is set via property, not with API
+    if (jmxPort < 0 && properties.containsKey(JMX_MANAGER_PORT)) {
+      // this will make sure we have all the missing properties, but it won't override
+      // the existing properties
+      withJMXManager(false);
+    }
+  }
+
   abstract void stopMember();
 
   @Override
@@ -78,6 +181,11 @@ public abstract class MemberStarterRule extends ExternalResource implements Memb
   }
 
   @Override
+  public int getHttpPort() {
+    return httpPort;
+  }
+
+  @Override
   public String getName() {
     return name;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
index 988f5d4..05e541a 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberVM.java
@@ -42,7 +42,7 @@ public class MemberVM<T extends Member> implements Member {
   }
 
   public T getMember() {
-    return member;
+    return (T) member;
   }
 
   @Override
@@ -61,6 +61,11 @@ public class MemberVM<T extends Member> implements Member {
   }
 
   @Override
+  public int getHttpPort() {
+    return member.getHttpPort();
+  }
+
+  @Override
   public String getName() {
     return member.getName();
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
index fb8630a..91a3af2 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/Server.java
@@ -24,4 +24,6 @@ public interface Server extends Member {
 
   CacheServer getServer();
 
+  int getEmbeddedLocatorPort();
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
index 0454340..fea2e9d 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
@@ -15,17 +15,17 @@
 
 package org.apache.geode.test.dunit.rules;
 
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_START;
-import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.ConfigurationProperties.LOG_FILE;
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.START_DEV_REST_API;
 
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 
 import java.io.File;
@@ -36,21 +36,14 @@ import java.util.Properties;
 /**
  * This is a rule to start up a server in your current VM. It's useful for your Integration Tests.
  *
- * You can create this rule either with a property or without a property. If created with a
- * property, The rule will automatically start the server for you with the properties given.
- *
- * If created without a property, the rule won't start the server until you specicially call one of
- * the startServer function.
- *
- * Either way, the rule will handle properly stopping the server for you.
- *
  * If you need a rule to start a server/locator in different VMs for Distributed tests, You should
  * use {@link LocatorServerStartupRule}.
  */
-public class ServerStarterRule extends MemberStarterRule implements Server {
+public class ServerStarterRule extends MemberStarterRule<ServerStarterRule> implements Server {
 
   private transient Cache cache;
   private transient CacheServer server;
+  private int embeddedLocatorPort = -1;
 
   /**
    * Default constructor, if used, the rule will create a temporary folder as the server's working
@@ -60,12 +53,12 @@ public class ServerStarterRule extends MemberStarterRule implements Server {
 
   /**
    * if constructed this way, the rule won't be deleting the workingDir after the test is done. It's
-   * up to the caller's responsibility to delete it.
+   * the caller's responsibility to delete it.
    * 
    * @param workingDir: the working dir this server should be writing the artifacts to.
    */
   public ServerStarterRule(File workingDir) {
-    this.workingDir = workingDir;
+    super(workingDir);
   }
 
   public Cache getCache() {
@@ -91,62 +84,47 @@ public class ServerStarterRule extends MemberStarterRule implements Server {
     }
   }
 
-  public ServerStarterRule startServer() {
-    return startServer(new Properties(), -1, false);
+  public ServerStarterRule withEmbeddedLocator() {
+    embeddedLocatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    properties.setProperty("start-locator", "localhost[" + embeddedLocatorPort + "]");
+    return this;
   }
 
-  public ServerStarterRule startServer(int locatorPort) {
-    return startServer(new Properties(), locatorPort, false);
+  public ServerStarterRule withRestService() {
+    return withRestService(false);
   }
 
-  public ServerStarterRule startServer(int locatorPort, boolean pdxPersistent) {
-    return startServer(new Properties(), locatorPort, pdxPersistent);
+  public ServerStarterRule withRestService(boolean useDefaultPort) {
+    properties.setProperty(START_DEV_REST_API, "true");
+    properties.setProperty(HTTP_SERVICE_BIND_ADDRESS, "localhost");
+    if (!useDefaultPort) {
+      httpPort = AvailablePortHelper.getRandomAvailableTCPPort();
+      properties.setProperty(HTTP_SERVICE_PORT, httpPort + "");
+    }
+    return this;
   }
 
-  public ServerStarterRule startServer(Properties properties) {
-    return startServer(properties, -1, false);
+  public ServerStarterRule startServer() {
+    return startServer(false);
   }
 
-  public ServerStarterRule startServer(Properties properties, int locatorPort) {
-    return startServer(properties, locatorPort, false);
+  public ServerStarterRule createRegion(RegionShortcut type, String name) {
+    cache.createRegionFactory(type).create(name);
+    return this;
   }
 
-  public ServerStarterRule startServer(Properties properties, int locatorPort,
-      boolean pdxPersistent) {
-    if (properties == null) {
-      properties = new Properties();
-    }
-    if (!properties.containsKey(NAME)) {
-      properties.setProperty(NAME, "server");
-    }
-    name = properties.getProperty(NAME);
-    if (!properties.containsKey(LOG_FILE)) {
-      properties.setProperty(LOG_FILE, new File(name + ".log").getAbsolutePath().toString());
-    }
-
-    if (locatorPort > 0) {
-      properties.setProperty(LOCATORS, "localhost[" + locatorPort + "]");
-    }
-    if (!properties.containsKey(MCAST_PORT)) {
-      properties.setProperty(MCAST_PORT, "0");
-    }
+  public ServerStarterRule startServer(Properties properties, int locatorPort) {
+    return withProperties(properties).withConnectionToLocator(locatorPort).startServer();
+  }
 
-    if (!properties.containsKey(LOCATORS)) {
-      properties.setProperty(LOCATORS, "");
-    }
-    if (properties.containsKey(JMX_MANAGER_PORT)) {
-      jmxPort = Integer.parseInt(properties.getProperty(JMX_MANAGER_PORT));
-      if (jmxPort > 0) {
-        if (!properties.containsKey(JMX_MANAGER))
-          properties.put(JMX_MANAGER, "true");
-        if (!properties.containsKey(JMX_MANAGER_START))
-          properties.put(JMX_MANAGER_START, "true");
-      }
-    }
-    CacheFactory cf = new CacheFactory(properties);
+  public ServerStarterRule startServer(boolean pdxPersistent) {
+    normalizeProperties();
+    CacheFactory cf = new CacheFactory(this.properties);
     cf.setPdxReadSerialized(pdxPersistent);
     cf.setPdxPersistent(pdxPersistent);
     cache = cf.create();
+    DistributionConfig config =
+        ((InternalDistributedSystem) cache.getDistributedSystem()).getConfig();
     server = cache.addCacheServer();
     server.setPort(0);
     try {
@@ -155,6 +133,13 @@ public class ServerStarterRule extends MemberStarterRule implements Server {
       throw new RuntimeException("unable to start server", e);
     }
     memberPort = server.getPort();
+    jmxPort = config.getJmxManagerPort();
+    httpPort = config.getHttpServicePort();
     return this;
   }
+
+  public int getEmbeddedLocatorPort() {
+    return embeddedLocatorPort;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDUnitTest.java
new file mode 100644
index 0000000..11db8cf
--- /dev/null
+++ b/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDUnitTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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.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.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
+
+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.ClientCacheFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.client.Pool;
+import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.client.internal.ProxyCache;
+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.security.templates.UserPasswordAuthInit;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Properties;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class CQClientAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "testRegion";
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .withProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName()).startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
+  }
+
+  @Test
+  public void testPostProcess() {
+    String query = "select * from /" + REGION_NAME;
+    client1.invoke(() -> {
+      Properties props = new Properties();
+      props.setProperty(LOCATORS, "");
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(SECURITY_CLIENT_AUTH_INIT,
+          UserPasswordAuthInit.class.getName() + ".create");
+      ClientCacheFactory factory = new ClientCacheFactory(props);
+
+      factory.addPoolServer("localhost", server.getPort());
+      factory.setPoolThreadLocalConnections(false);
+      factory.setPoolMinConnections(5);
+      factory.setPoolSubscriptionEnabled(true);
+      factory.setPoolMultiuserAuthentication(true);
+
+
+      ClientCache clientCache = factory.create();
+      Region region =
+          clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      Pool pool = PoolManager.find(region);
+
+      Properties userProps = new Properties();
+      userProps.setProperty("security-username", "super-user");
+      userProps.setProperty("security-password", "1234567");
+      ProxyCache cache =
+          (ProxyCache) clientCache.createAuthenticatedView(userProps, pool.getName());
+
+      QueryService qs = cache.getQueryService();
+
+      CqAttributesFactory cqAttributesFactory = new CqAttributesFactory();
+
+      CqAttributes cqa = cqAttributesFactory.create();
+
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa, true);
+      cq.execute();
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java b/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java
deleted file mode 100644
index ce1782b..0000000
--- a/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java
+++ /dev/null
@@ -1,92 +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.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_POST_PROCESSOR;
-
-import org.apache.geode.cache.Region;
-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.Pool;
-import org.apache.geode.cache.client.PoolManager;
-import org.apache.geode.cache.client.internal.ProxyCache;
-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.security.templates.UserPasswordAuthInit;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.Properties;
-
-@Category({DistributedTest.class, SecurityTest.class})
-public class CQClientAuthDunitTest extends AbstractSecureServerDUnitTest {
-
-  public Properties getProperties() {
-    Properties properties = super.getProperties();
-    properties.setProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName());
-    return properties;
-  }
-
-  @Test
-  public void testPostProcess() {
-    String query = "select * from /AuthRegion";
-    client1.invoke(() -> {
-      Properties props = new Properties();
-      props.setProperty(LOCATORS, "");
-      props.setProperty(MCAST_PORT, "0");
-      props.setProperty(SECURITY_CLIENT_AUTH_INIT,
-          UserPasswordAuthInit.class.getName() + ".create");
-      ClientCacheFactory factory = new ClientCacheFactory(props);
-
-      factory.addPoolServer("localhost", this.serverPort);
-      factory.setPoolThreadLocalConnections(false);
-      factory.setPoolMinConnections(5);
-      factory.setPoolSubscriptionEnabled(true);
-      factory.setPoolMultiuserAuthentication(true);
-
-
-      ClientCache clientCache = factory.create();
-      Region region =
-          clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
-      Pool pool = PoolManager.find(region);
-
-      Properties userProps = new Properties();
-      userProps.setProperty("security-username", "super-user");
-      userProps.setProperty("security-password", "1234567");
-      ProxyCache cache =
-          (ProxyCache) clientCache.createAuthenticatedView(userProps, pool.getName());
-
-      QueryService qs = cache.getQueryService();
-
-      CqAttributesFactory cqAttributesFactory = new CqAttributesFactory();
-
-      CqAttributes cqa = cqAttributesFactory.create();
-
-      // Create the CqQuery
-      CqQuery cq = qs.newCq("CQ1", query, cqa, true);
-      cq.execute();
-    });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-cq/src/test/java/org/apache/geode/security/CQPDXPostProcessorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/CQPDXPostProcessorDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/security/CQPDXPostProcessorDUnitTest.java
index 0bafb68..6c897d1 100644
--- a/geode-cq/src/test/java/org/apache/geode/security/CQPDXPostProcessorDUnitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/security/CQPDXPostProcessorDUnitTest.java
@@ -15,23 +15,15 @@
 
 package org.apache.geode.security;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-
-import org.awaitility.Awaitility;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 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;
@@ -42,19 +34,38 @@ import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.CqResults;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.cache.query.internal.cq.CqListenerImpl;
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.pdx.SimpleClass;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+import org.awaitility.Awaitility;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
 
 @Category({DistributedTest.class, SecurityTest.class})
 @RunWith(Parameterized.class)
 @Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class CQPDXPostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
+public class CQPDXPostProcessorDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  private boolean pdxPersistent = false;
   private static byte[] BYTES = {1, 0};
-  private static int jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
   @Parameterized.Parameters
   public static Collection<Object[]> parameters() {
@@ -62,17 +73,14 @@ public class CQPDXPostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
     return Arrays.asList(params);
   }
 
-  public Properties getProperties() {
-    Properties properties = super.getProperties();
-    properties.setProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName());
-    properties.setProperty("security-pdx", pdxPersistent + "");
-    properties.setProperty(JMX_MANAGER_PORT, jmxPort + "");
-    return properties;
-  }
-
-  public Map<String, String> getData() {
-    return new HashMap();
-  }
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .withProperty(SECURITY_POST_PROCESSOR, PDXPostProcessor.class.getName())
+          .withProperty("security-pdx", pdxPersistent + "").startServer()
+          .createRegion(RegionShortcut.REPLICATE, REGION_NAME);
 
   public CQPDXPostProcessorDUnitTest(boolean pdxPersistent) {
     this.pdxPersistent = pdxPersistent;
@@ -80,10 +88,10 @@ public class CQPDXPostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
 
   @Test
   public void testCQ() {
-    String query = "select * from /AuthRegion";
+    String query = "select * from /" + REGION_NAME;
     client1.invoke(() -> {
-      ClientCache cache = createClientCache("super-user", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
 
       Pool pool = PoolManager.find(region);
       QueryService qs = pool.getQueryService();
@@ -111,8 +119,8 @@ public class CQPDXPostProcessorDUnitTest extends AbstractSecureServerDUnitTest {
     });
 
     client2.invoke(() -> {
-      ClientCache cache = createClientCache("authRegionUser", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
       region.put("key1", new SimpleClass(1, (byte) 1));
       region.put("key2", BYTES);
     });

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-cq/src/test/java/org/apache/geode/security/CQPostProcessorDunitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/CQPostProcessorDunitTest.java b/geode-cq/src/test/java/org/apache/geode/security/CQPostProcessorDunitTest.java
index 8510dce..26eff48 100644
--- a/geode-cq/src/test/java/org/apache/geode/security/CQPostProcessorDunitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/security/CQPostProcessorDunitTest.java
@@ -15,15 +15,15 @@
 
 package org.apache.geode.security;
 
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_POST_PROCESSOR;
-import static org.junit.Assert.*;
-
-import java.util.Properties;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import static org.apache.geode.security.SecurityTestUtil.createClientCache;
+import static org.apache.geode.security.SecurityTestUtil.createProxyRegion;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 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;
@@ -34,25 +34,47 @@ import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.CqResults;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.cache.query.internal.cq.CqListenerImpl;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 @Category({DistributedTest.class, SecurityTest.class})
-public class CQPostProcessorDunitTest extends AbstractSecureServerDUnitTest {
-
-  public Properties getProperties() {
-    Properties properties = super.getProperties();
-    properties.setProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName());
-    return properties;
+public class CQPostProcessorDunitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .withProperty(SECURITY_POST_PROCESSOR, TestPostProcessor.class.getName()).startServer();
+
+  @Before
+  public void before() throws Exception {
+    Region region =
+        server.getCache().createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+    for (int i = 0; i < 5; i++) {
+      region.put("key" + i, "value" + i);
+    }
   }
 
-
   @Test
   public void testPostProcess() {
     String query = "select * from /AuthRegion";
     client1.invoke(() -> {
-      ClientCache cache = createClientCache("super-user", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
 
 
 
@@ -85,8 +107,8 @@ public class CQPostProcessorDunitTest extends AbstractSecureServerDUnitTest {
     });
 
     client2.invoke(() -> {
-      ClientCache cache = createClientCache("authRegionUser", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
+      ClientCache cache = createClientCache("authRegionUser", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
       region.put("key6", "value6");
     });
 

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDUnitTest.java
new file mode 100644
index 0000000..18749d4
--- /dev/null
+++ b/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDUnitTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.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.apache.geode.cache.RegionShortcut;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+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.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.CqEvent;
+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.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({DistributedTest.class, SecurityTest.class})
+public class ClientQueryAuthDUnitTest extends JUnit4DistributedTestCase {
+
+  private static String REGION_NAME = "AuthRegion";
+  final Host host = Host.getHost(0);
+  final VM client1 = host.getVM(1);
+  final VM client2 = host.getVM(2);
+  final VM client3 = host.getVM(3);
+
+  @Rule
+  public ServerStarterRule server =
+      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
+          .withProperty(TestSecurityManager.SECURITY_JSON,
+              "org/apache/geode/management/internal/security/clientServer.json")
+          .startServer().createRegion(RegionShortcut.REPLICATE, REGION_NAME);
+
+  @Test
+  public void testQuery() {
+    client1.invoke(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+      final Region region = createProxyRegion(cache, REGION_NAME);
+
+      String query = "select * from /AuthRegion";
+      assertNotAuthorized(() -> region.query(query), "DATA:READ:AuthRegion");
+
+      Pool pool = PoolManager.find(region);
+      assertNotAuthorized(() -> pool.getQueryService().newQuery(query).execute(),
+          "DATA:READ:AuthRegion");
+    });
+  }
+
+  @Test
+  public void testCQ() {
+    String query = "select * from /AuthRegion";
+    client1.invoke(() -> {
+      ClientCache cache = createClientCache("stranger", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+      Pool pool = PoolManager.find(region);
+      QueryService qs = pool.getQueryService();
+
+      CqAttributes cqa = new CqAttributesFactory().create();
+
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa);
+
+      assertNotAuthorized(() -> cq.executeWithInitialResults(), "DATA:READ:AuthRegion");
+      assertNotAuthorized(() -> cq.execute(), "DATA:READ:AuthRegion");
+
+      assertNotAuthorized(() -> cq.close(), "DATA:MANAGE");
+    });
+
+    client2.invoke(() -> {
+      ClientCache cache = createClientCache("authRegionReader", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+      Pool pool = PoolManager.find(region);
+      QueryService qs = pool.getQueryService();
+
+      CqAttributes cqa = new CqAttributesFactory().create();
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa);
+      cq.execute();
+
+      assertNotAuthorized(() -> cq.stop(), "DATA:MANAGE");
+      assertNotAuthorized(() -> qs.getAllDurableCqsFromServer(), "CLUSTER:READ");
+    });
+
+    client3.invoke(() -> {
+      ClientCache cache = createClientCache("super-user", "1234567", server.getPort());
+      Region region = createProxyRegion(cache, REGION_NAME);
+      Pool pool = PoolManager.find(region);
+      QueryService qs = pool.getQueryService();
+
+      CqAttributesFactory factory = new CqAttributesFactory();
+      factory.addCqListener(new CqListener() {
+        @Override
+        public void onEvent(final CqEvent aCqEvent) {
+          System.out.println(aCqEvent);
+        }
+
+        @Override
+        public void onError(final CqEvent aCqEvent) {
+
+        }
+
+        @Override
+        public void close() {
+
+        }
+      });
+
+
+      CqAttributes cqa = factory.create();
+
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa);
+      System.out.println("query result: " + cq.executeWithInitialResults());
+
+      cq.stop();
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDistributedTest.java b/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDistributedTest.java
deleted file mode 100644
index 9d82ae9..0000000
--- a/geode-cq/src/test/java/org/apache/geode/security/ClientQueryAuthDistributedTest.java
+++ /dev/null
@@ -1,121 +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.security;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.Region;
-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.CqEvent;
-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.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
-@Category({DistributedTest.class, SecurityTest.class})
-public class ClientQueryAuthDistributedTest extends AbstractSecureServerDUnitTest {
-
-  @Test
-  public void testQuery() {
-    client1.invoke(() -> {
-      ClientCache cache = createClientCache("stranger", "1234567", serverPort);
-      final Region region = cache.getRegion(REGION_NAME);
-
-      String query = "select * from /AuthRegion";
-      assertNotAuthorized(() -> region.query(query), "DATA:READ:AuthRegion");
-
-      Pool pool = PoolManager.find(region);
-      assertNotAuthorized(() -> pool.getQueryService().newQuery(query).execute(),
-          "DATA:READ:AuthRegion");
-    });
-  }
-
-  @Test
-  public void testCQ() {
-    String query = "select * from /AuthRegion";
-    client1.invoke(() -> {
-      ClientCache cache = createClientCache("stranger", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
-      Pool pool = PoolManager.find(region);
-      QueryService qs = pool.getQueryService();
-
-      CqAttributes cqa = new CqAttributesFactory().create();
-
-      // Create the CqQuery
-      CqQuery cq = qs.newCq("CQ1", query, cqa);
-
-      assertNotAuthorized(() -> cq.executeWithInitialResults(), "DATA:READ:AuthRegion");
-      assertNotAuthorized(() -> cq.execute(), "DATA:READ:AuthRegion");
-
-      assertNotAuthorized(() -> cq.close(), "DATA:MANAGE");
-    });
-
-    client2.invoke(() -> {
-      ClientCache cache = createClientCache("authRegionReader", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
-      Pool pool = PoolManager.find(region);
-      QueryService qs = pool.getQueryService();
-
-      CqAttributes cqa = new CqAttributesFactory().create();
-      // Create the CqQuery
-      CqQuery cq = qs.newCq("CQ1", query, cqa);
-      cq.execute();
-
-      assertNotAuthorized(() -> cq.stop(), "DATA:MANAGE");
-      assertNotAuthorized(() -> qs.getAllDurableCqsFromServer(), "CLUSTER:READ");
-    });
-
-    client3.invoke(() -> {
-      ClientCache cache = createClientCache("super-user", "1234567", serverPort);
-      Region region = cache.getRegion(REGION_NAME);
-      Pool pool = PoolManager.find(region);
-      QueryService qs = pool.getQueryService();
-
-      CqAttributesFactory factory = new CqAttributesFactory();
-      factory.addCqListener(new CqListener() {
-        @Override
-        public void onEvent(final CqEvent aCqEvent) {
-          System.out.println(aCqEvent);
-        }
-
-        @Override
-        public void onError(final CqEvent aCqEvent) {
-
-        }
-
-        @Override
-        public void close() {
-
-        }
-      });
-
-
-      CqAttributes cqa = factory.create();
-
-      // Create the CqQuery
-      CqQuery cq = qs.newCq("CQ1", query, cqa);
-      System.out.println("query result: " + cq.executeWithInitialResults());
-
-      cq.stop();
-    });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
index 420f2dd..90f16ea 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/ExportLogsOverHttpIntegrationTest.java
@@ -24,7 +24,7 @@ public class ExportLogsOverHttpIntegrationTest extends ExportLogsIntegrationTest
 
   @Override
   protected void connect() throws Exception {
-    gfsh.connectAndVerify(httpPort, GfshShellConnectionRule.PortType.http);
+    gfsh.connectAndVerify(locator.getHttpPort(), GfshShellConnectionRule.PortType.http);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java
index 22ca6e3..f63cf12 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/cli/commands/QueryNamesOverHttpDUnitTest.java
@@ -15,13 +15,9 @@
 
 package org.apache.geode.management.internal.cli.commands;
 
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
-import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_PORT;
-import static org.apache.geode.distributed.ConfigurationProperties.JMX_MANAGER_PORT;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 
-import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 import org.apache.geode.management.internal.web.domain.Link;
 import org.apache.geode.management.internal.web.domain.LinkIndex;
@@ -35,7 +31,6 @@ import org.junit.experimental.categories.Category;
 
 import java.net.URI;
 import java.util.HashMap;
-import java.util.Properties;
 import java.util.Set;
 import javax.management.ObjectName;
 import javax.management.Query;
@@ -43,27 +38,16 @@ import javax.management.QueryExp;
 
 @Category(IntegrationTest.class)
 public class QueryNamesOverHttpDUnitTest {
-  protected static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
-  protected static int jmxPort = ports[0];
-  protected static int httpPort = ports[1];
-
-  protected static Properties locatorProps = new Properties() {
-    {
-      setProperty(HTTP_SERVICE_BIND_ADDRESS, "localhost");
-      setProperty(HTTP_SERVICE_PORT, httpPort + "");
-      setProperty(JMX_MANAGER_PORT, jmxPort + "");
-    }
-  };
-
   @Rule
-  public LocatorStarterRule locatorRule = new LocatorStarterRule().startLocator(locatorProps);
+  public LocatorStarterRule locatorRule = new LocatorStarterRule().withJMXManager().startLocator();
 
   @Test
   public void testQueryNameOverHttp() throws Exception {
 
     LinkIndex links = new LinkIndex();
     links.add(new Link("mbean-query",
-        new URI("http://localhost:" + httpPort + "/gemfire/v1/mbean/query"), HttpMethod.POST));
+        new URI("http://localhost:" + locatorRule.getHttpPort() + "/gemfire/v1/mbean/query"),
+        HttpMethod.POST));
     RestHttpOperationInvoker invoker =
         new RestHttpOperationInvoker(links, mock(Gfsh.class), new HashMap<>());
 

http://git-wip-us.apache.org/repos/asf/geode/blob/69dbc10f/geode-web/src/test/java/org/apache/geode/management/internal/security/GfshCommandsOverHttpSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/org/apache/geode/management/internal/security/GfshCommandsOverHttpSecurityTest.java b/geode-web/src/test/java/org/apache/geode/management/internal/security/GfshCommandsOverHttpSecurityTest.java
index 5449568..7fd839c 100644
--- a/geode-web/src/test/java/org/apache/geode/management/internal/security/GfshCommandsOverHttpSecurityTest.java
+++ b/geode-web/src/test/java/org/apache/geode/management/internal/security/GfshCommandsOverHttpSecurityTest.java
@@ -24,6 +24,7 @@ import org.junit.experimental.categories.Category;
 @Category({IntegrationTest.class, SecurityTest.class})
 public class GfshCommandsOverHttpSecurityTest extends GfshCommandsSecurityTest {
   public GfshCommandsOverHttpSecurityTest() {
-    gfshConnection = new GfshShellConnectionRule(httpPort, GfshShellConnectionRule.PortType.http);
+    gfshConnection = new GfshShellConnectionRule(serverStarter.getHttpPort(),
+        GfshShellConnectionRule.PortType.http);
   }
 }