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 2016/03/31 17:23:23 UTC

[60/64] [abbrv] incubator-geode git commit: GEODE-693: refactor security dunit tests

GEODE-693: refactor security dunit tests

* GEODE-1114: remove com.gemstone.gemfire.internal.util.Callable
* convert security dunit tests to JUnit 4
* use RetryRule on ClientPostAuthorizationDUnitTest.testAllPostOps
* convert public variables and methods to private and/or protected
* convert many static variables and methods to instance


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/22ca5ef8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/22ca5ef8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/22ca5ef8

Branch: refs/heads/feature/GEODE-17-2
Commit: 22ca5ef829fee20024d44264b3839ee745916975
Parents: 48af841
Author: Kirk Lund <kl...@pivotal.io>
Authored: Wed Mar 30 15:13:17 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Wed Mar 30 15:25:52 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/EntriesMap.java      |    5 +-
 .../gemfire/internal/util/Callable.java         |   47 -
 .../cli/ClasspathScanLoadHelperJUnitTest.java   |   85 +-
 .../security/ClientAuthenticationDUnitTest.java |  889 +-------
 .../ClientAuthenticationPart2DUnitTest.java     |   96 +-
 .../security/ClientAuthenticationTestCase.java  |  590 ++++++
 .../security/ClientAuthenticationTestUtils.java |   93 +
 .../security/ClientAuthorizationDUnitTest.java  | 1094 +++++-----
 .../security/ClientAuthorizationTestBase.java   | 1381 -------------
 .../security/ClientAuthorizationTestCase.java   | 1323 ++++++++++++
 .../security/ClientMultiUserAuthzDUnitTest.java |  590 ++----
 .../DeltaClientAuthorizationDUnitTest.java      |  396 ++--
 .../DeltaClientPostAuthorizationDUnitTest.java  |  614 ++----
 .../security/P2PAuthenticationDUnitTest.java    |  667 +++---
 .../gemfire/security/SecurityTestUtil.java      | 1918 ------------------
 .../gemfire/security/SecurityTestUtils.java     | 1683 +++++++++++++++
 .../ClientAuthorizationTwoDUnitTest.java        |  223 +-
 .../security/ClientAuthzObjectModDUnitTest.java |  466 ++---
 .../ClientCQPostAuthorizationDUnitTest.java     |  449 ++--
 .../ClientPostAuthorizationDUnitTest.java       |  461 ++---
 .../gemfire/security/MultiUserAPIDUnitTest.java |  314 +++
 .../MultiUserDurableCQAuthzDUnitTest.java       |  387 ++++
 .../gemfire/security/MultiuserAPIDUnitTest.java |  381 ----
 .../MultiuserDurableCQAuthzDUnitTest.java       |  475 -----
 24 files changed, 6371 insertions(+), 8256 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22ca5ef8/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntriesMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntriesMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntriesMap.java
index 74697ea..257458f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntriesMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntriesMap.java
@@ -20,10 +20,9 @@ package com.gemstone.gemfire.internal.cache;
 import java.util.*;
 import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.internal.InternalStatisticsDisabledException;
-import com.gemstone.gemfire.internal.util.Callable;
-//import com.gemstone.gemfire.util.concurrent.ConcurrentMap;
+
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
-//import com.gemstone.gemfire.util.concurrent.locks.*;
 import com.gemstone.gemfire.distributed.internal.DM;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22ca5ef8/geode-core/src/main/java/com/gemstone/gemfire/internal/util/Callable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/Callable.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/Callable.java
deleted file mode 100755
index 7799224..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/Callable.java
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Written by Doug Lea with assistance from members of JCP JSR-166
- * Expert Group and released to the public domain. Use, modify, and
- * redistribute this code in any way without acknowledgement.
- */
-
-package com.gemstone.gemfire.internal.util;
-
-/**
- * A task that returns a result and may throw an exception.
- * Implementors define a single method with no arguments called
- * <tt>call</tt>.
- *
- * <p>The <tt>Callable</tt> interface is similar to {@link
- * java.lang.Runnable}, in that both are designed for classes whose
- * instances are potentially executed by another thread.  A
- * <tt>Runnable</tt>, however, does not return a result and cannot
- * throw a checked exception.
- *
- *
- * @version based on JSR166 Callable version 1.6.
- */
-public interface Callable {
-    /**
-     * Computes a result, or throws an exception if unable to do so.
-     *
-     * @return computed result
-     * @throws Exception if unable to compute a result
-     */
-    Object call() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22ca5ef8/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ClasspathScanLoadHelperJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ClasspathScanLoadHelperJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ClasspathScanLoadHelperJUnitTest.java
index 4476cd6..79a0f09 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ClasspathScanLoadHelperJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/ClasspathScanLoadHelperJUnitTest.java
@@ -16,14 +16,12 @@
  */
 package com.gemstone.gemfire.management.internal.cli;
 
+import static org.junit.Assert.*;
+
 import java.io.IOException;
 import java.util.Set;
 
-import org.junit.experimental.categories.Category;
-
-import junit.framework.TestCase;
-
-import com.gemstone.gemfire.internal.util.Callable;
+import com.gemstone.gemfire.internal.util.Versionable;
 import com.gemstone.gemfire.management.internal.cli.domain.AbstractImpl;
 import com.gemstone.gemfire.management.internal.cli.domain.Impl1;
 import com.gemstone.gemfire.management.internal.cli.domain.Impl12;
@@ -31,61 +29,52 @@ import com.gemstone.gemfire.management.internal.cli.domain.Interface1;
 import com.gemstone.gemfire.management.internal.cli.domain.Interface2;
 import com.gemstone.gemfire.management.internal.cli.util.ClasspathScanLoadHelper;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-/**
- */
 @Category(UnitTest.class)
-public class ClasspathScanLoadHelperJUnitTest  extends TestCase{
+public class ClasspathScanLoadHelperJUnitTest {
   
   private static final String PACKAGE_NAME = "com.gemstone.gemfire.management.internal.cli.domain";
   private static final String WRONG_PACKAGE_NAME = "com.gemstone.gemfire.management.internal.cli.domain1";
   private static final Class<?> INTERFACE1 = Interface1.class;
-  private static final Class<?> NO_IMPL_INTERFACE = Callable.class;
+  private static final Class<?> NO_IMPL_INTERFACE = Versionable.class;
   private static final Class<?> INTERFACE2 = Interface2.class;
   private static final Class<?> IMPL1 = Impl1.class;
   private static final Class<?> IMPL2 = Impl12.class;
   private static final Class<?> ABSTRACT_IMPL = AbstractImpl.class;
   
+  @Test
+  public void testLoadAndGet() throws Exception {
+    Set<Class<?>> classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE1, true);
+    assertEquals(2, classLoaded.size());
+    assertTrue(classLoaded.contains(IMPL1));
+    assertTrue(classLoaded.contains(IMPL2));
+    //impl1 and impl12
+
+    classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE1, false);
+    assertEquals(4, classLoaded.size());
+    assertTrue(classLoaded.contains(IMPL1));
+    assertTrue(classLoaded.contains(IMPL2));
+    assertTrue(classLoaded.contains(ABSTRACT_IMPL));
+    assertTrue(classLoaded.contains(INTERFACE1));
+
+    classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE2, false);
+    assertEquals(2, classLoaded.size());
+    assertTrue(classLoaded.contains(IMPL2));
+    assertTrue(classLoaded.contains(INTERFACE2));
+
+    classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE2, true);
+    assertEquals(1, classLoaded.size());
+    assertTrue(classLoaded.contains(IMPL2));
+
+    classLoaded = ClasspathScanLoadHelper.loadAndGet(WRONG_PACKAGE_NAME, INTERFACE2, true);
+    assertEquals(0, classLoaded.size());
 
-  public void testloadAndGet(){
+    classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, NO_IMPL_INTERFACE, true);
+    assertEquals(0, classLoaded.size());
 
-    try {
-      Set<Class<?>> classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE1, true);
-      assertEquals(2, classLoaded.size());
-      assertTrue(classLoaded.contains(IMPL1));
-      assertTrue(classLoaded.contains(IMPL2));
-      //impl1 and impl12
-      
-      classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE1, false);
-      assertEquals(4, classLoaded.size());
-      assertTrue(classLoaded.contains(IMPL1));
-      assertTrue(classLoaded.contains(IMPL2));
-      assertTrue(classLoaded.contains(ABSTRACT_IMPL));
-      assertTrue(classLoaded.contains(INTERFACE1));
-      
-      classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE2, false);
-      assertEquals(2, classLoaded.size());      
-      assertTrue(classLoaded.contains(IMPL2));      
-      assertTrue(classLoaded.contains(INTERFACE2));
-      
-      classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, INTERFACE2, true);      
-      assertEquals(1, classLoaded.size());      
-      assertTrue(classLoaded.contains(IMPL2));
-      
-      classLoaded = ClasspathScanLoadHelper.loadAndGet(WRONG_PACKAGE_NAME, INTERFACE2, true);
-      assertEquals(0, classLoaded.size());      
-      
-      classLoaded = ClasspathScanLoadHelper.loadAndGet(PACKAGE_NAME, NO_IMPL_INTERFACE, true);
-      assertEquals(0, classLoaded.size());
-      
-      classLoaded = ClasspathScanLoadHelper.loadAndGet(WRONG_PACKAGE_NAME, NO_IMPL_INTERFACE, true);
-      assertEquals(0, classLoaded.size());
-      
-      
-    } catch (ClassNotFoundException e) {
-      fail("Error loading class" + e);
-    } catch (IOException e) {
-      fail("Error loading class" + e);
-    }   
+    classLoaded = ClasspathScanLoadHelper.loadAndGet(WRONG_PACKAGE_NAME, NO_IMPL_INTERFACE, true);
+    assertEquals(0, classLoaded.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22ca5ef8/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
index 8741f58..ea83a66 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
@@ -18,25 +18,10 @@
  */
 package com.gemstone.gemfire.security;
 
-import java.io.IOException;
-import java.util.Properties;
-
-import javax.net.ssl.SSLException;
-import javax.net.ssl.SSLHandshakeException;
-
-import com.gemstone.gemfire.security.generator.CredentialGenerator;
-import com.gemstone.gemfire.security.generator.CredentialGenerator.ClassCode;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.security.generator.DummyCredentialGenerator;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * Test for authentication from client to server. This tests for both valid and
@@ -45,857 +30,57 @@ import com.gemstone.gemfire.test.dunit.Wait;
  * 
  * @since 5.5
  */
-public class ClientAuthenticationDUnitTest extends DistributedTestCase {
-
-  /** constructor */
-  public ClientAuthenticationDUnitTest(String name) {
-    super(name);
-  }
-
-  private VM server1 = null;
-
-  private VM server2 = null;
-
-  private VM client1 = null;
-
-  private VM client2 = null;
-
-  private static final String[] serverExpectedExceptions = {
-      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[] clientExpectedExceptions = {
-      AuthenticationRequiredException.class.getName(),
-      AuthenticationFailedException.class.getName(),
-      SSLHandshakeException.class.getName() };
-
-  @Override
-  public final void postSetUp() throws Exception {
-    final Host host = Host.getHost(0);
-    server1 = host.getVM(0);
-    server2 = host.getVM(1);
-    client1 = host.getVM(2);
-    client2 = host.getVM(3);
-    
-    IgnoredException.addIgnoredException("Connection refused: connect");
-
-    server1.invoke(() -> SecurityTestUtil.registerExpectedExceptions( serverExpectedExceptions ));
-    server2.invoke(() -> SecurityTestUtil.registerExpectedExceptions( serverExpectedExceptions ));
-    client1.invoke(() -> SecurityTestUtil.registerExpectedExceptions( clientExpectedExceptions ));
-    client2.invoke(() -> SecurityTestUtil.registerExpectedExceptions( clientExpectedExceptions ));
-  }
-
-  // Region: Utility and static functions invoked by the tests
-
-  public static Integer createCacheServer(Object dsPort, Object locatorString,
-      Object authenticator, Object extraProps, Object javaProps) {
-
-    Properties authProps;
-    if (extraProps == null) {
-      authProps = new Properties();
-    }
-    else {
-      authProps = (Properties)extraProps;
-    }
-    if (authenticator != null) {
-      authProps.setProperty(
-          DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator
-              .toString());
-    }
-    return SecurityTestUtil.createCacheServer(authProps, javaProps,
-        (Integer)dsPort, (String)locatorString, null, new Integer(
-            SecurityTestUtil.NO_EXCEPTION));
-  }
-
-  public static void createCacheServer(Object dsPort, Object locatorString,
-      Integer serverPort, Object authenticator, Object extraProps,
-      Object javaProps) {
+@Category(DistributedTest.class)
+public class ClientAuthenticationDUnitTest extends ClientAuthenticationTestCase {
 
-    Properties authProps;
-    if (extraProps == null) {
-      authProps = new Properties();
-    }
-    else {
-      authProps = (Properties)extraProps;
-    }
-    if (authenticator != null) {
-      authProps.setProperty(
-          DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, authenticator
-              .toString());
-    }
-    SecurityTestUtil.createCacheServer(authProps, javaProps, (Integer)dsPort,
-        (String)locatorString, serverPort, new Integer(
-            SecurityTestUtil.NO_EXCEPTION));
+  @Test
+  public void testValidCredentials() throws Exception {
+    doTestValidCredentials(false);
   }
 
-  private static void createCacheClient(Object authInit, Properties authProps,
-      Properties javaProps, Integer[] ports, Object numConnections,
-      Boolean multiUserMode, Boolean subscriptionEnabled, Integer expectedResult) {
-
-    String authInitStr = (authInit == null ? null : authInit.toString());
-    SecurityTestUtil.createCacheClient(authInitStr, authProps, javaProps,
-        ports, (Integer)numConnections, Boolean.FALSE,
-        multiUserMode.toString(), subscriptionEnabled, expectedResult);
-  }
-
-  public static void createCacheClient(Object authInit, Object authProps,
-      Object javaProps, Integer[] ports, Object numConnections,
-      Boolean multiUserMode, Integer expectedResult) {
-
-    createCacheClient(authInit, (Properties)authProps, (Properties)javaProps,
-        ports, numConnections, multiUserMode, Boolean.TRUE, expectedResult);
-  }
-
-  public static void createCacheClient(Object authInit, Object authProps,
-      Object javaProps, Integer port1, Object numConnections,
-      Integer expectedResult) {
-
-    createCacheClient(authInit, (Properties)authProps, (Properties)javaProps,
-        new Integer[] { port1 }, numConnections, Boolean.FALSE, Boolean.TRUE,
-        expectedResult);
+  @Test
+  public void testNoCredentials() throws Exception {
+    doTestNoCredentials(false);
   }
 
-  public static void createCacheClient(Object authInit, Object authProps,
-      Object javaProps, Integer port1, Integer port2, Object numConnections,
-      Integer expectedResult) {
-    createCacheClient(authInit, authProps, javaProps, port1, port2,
-        numConnections, Boolean.FALSE, expectedResult);
+  @Test
+  public void testInvalidCredentials() throws Exception {
+    doTestInvalidCredentials(false);
   }
 
-  public static void createCacheClient(Object authInit, Object authProps,
-      Object javaProps, Integer port1, Integer port2, Object numConnections,
-      Boolean multiUserMode, Integer expectedResult) {
-
-    createCacheClient(authInit, authProps, javaProps,
-        port1, port2, numConnections, multiUserMode, Boolean.TRUE,
-        expectedResult);
+  @Test
+  public void testInvalidAuthInit() throws Exception {
+    doTestInvalidAuthInit(false);
   }
 
-  public static void createCacheClient(Object authInit, Object authProps,
-      Object javaProps, Integer port1, Integer port2, Object numConnections,
-      Boolean multiUserMode, Boolean subscriptionEnabled,
-      Integer expectedResult) {
-
-    createCacheClient(authInit, (Properties)authProps, (Properties)javaProps,
-        new Integer[] { port1, port2 }, numConnections, multiUserMode,
-        subscriptionEnabled, expectedResult);
+  @Test
+  public void testNoAuthInitWithCredentials() throws Exception {
+    doTestNoAuthInitWithCredentials(false);
   }
 
-  public static void registerAllInterest() {
-
-    Region region = SecurityTestUtil.getCache().getRegion(
-        SecurityTestUtil.regionName);
-    assertNotNull(region);
-    region.registerInterestRegex(".*");
+  @Test
+  public void testInvalidAuthenticator() throws Exception {
+    doTestInvalidAuthenticator(false);
   }
 
-  // End Region: Utility and static functions invoked by the tests
-
-  // Region: Tests
-
-  public void testValidCredentials() {
-    itestValidCredentials(Boolean.FALSE);
-  }
-
-  public void itestValidCredentials(Boolean multiUser) {
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter().info(
-          "testValidCredentials: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testValidCredentials: Using authenticator: " + authenticator);
-      LogWriterUtils.getLogWriter().info("testValidCredentials: Using authinit: " + authInit);
-
-      // Start the servers
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = (Integer)server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, authenticator, extraProps,
-              javaProps ));
-      Integer port2 = (Integer)server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, authenticator, extraProps,
-              javaProps ));
-
-      // Start the clients with valid credentials
-      Properties credentials1 = gen.getValidCredentials(1);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testValidCredentials: For first client credentials: " + credentials1
-              + " : " + javaProps1);
-      Properties credentials2 = gen.getValidCredentials(2);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testValidCredentials: For second client credentials: "
-              + credentials2 + " : " + javaProps2);
-      createClientsNoException(multiUser, authInit, port1, port2, credentials1,
-          javaProps1, credentials2, javaProps2);
-
-      // Perform some put operations from client1
-      client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(2) ));
-
-      // Verify that the puts succeeded
-      client2.invoke(() -> SecurityTestUtil.doGets( new Integer(2) ));
-      
-      if (multiUser) {
-        client1.invoke(() -> SecurityTestUtil.doProxyCacheClose());
-        client2.invoke(() -> SecurityTestUtil.doProxyCacheClose());
-        client1.invoke(() -> SecurityTestUtil.doSimplePut("CacheClosedException"));
-        client2.invoke(() -> SecurityTestUtil.doSimpleGet("CacheClosedException"));
-      }
+  @Test
+  public void testNoAuthenticatorWithCredentials() throws Exception {
+    doTestNoAuthenticatorWithCredentials(false);
   }
 
-  public void testNoCredentials() {
-    itestNoCredentials(Boolean.FALSE);
+  @Test
+  public void testCredentialsWithFailover() throws Exception {
+    doTestCredentialsWithFailover(false);
   }
 
-  public void itestNoCredentials(Boolean multiUser) {
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter()
-          .info("testNoCredentials: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testNoCredentials: Using authenticator: " + authenticator);
-      LogWriterUtils.getLogWriter().info("testNoCredentials: Using authinit: " + authInit);
-
-      // Start the servers
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = createServer1(extraProps, javaProps, authenticator,
-          locPort1, locString);
-      Integer port2 = ((Integer)server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, authenticator, extraProps,
-              javaProps )));
-
-      // Start first client with valid credentials
-      Properties credentials1 = gen.getValidCredentials(1);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testNoCredentials: For first client credentials: " + credentials1
-              + " : " + javaProps1);
-      createClient1NoException(multiUser, authInit, port1, port2, credentials1,
-          javaProps1);
-
-      // Perform some put operations from client1
-      client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(2) ));
-
-      // Trying to create the region on client2 
-      if (gen.classCode().equals(ClassCode.SSL)) {
-        // For SSL the exception may not come since the server can close socket
-        // before handshake message is sent from client. However exception
-        // should come in any region operations.
-        client2
-            .invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, null, null, port1, port2, null, multiUser,
-                    new Integer(SecurityTestUtil.NO_EXCEPTION) ));
-        client2.invoke(() -> SecurityTestUtil.doPuts(
-            new Integer(2), new Integer(SecurityTestUtil.OTHER_EXCEPTION) ));
-      }
-      else {
-        client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, null, null, port1, port2,
-                null, multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-      }
-  }
-
-  public void testInvalidCredentials() {
-    itestInvalidCredentials(Boolean.FALSE);
-  }
-
-  public void itestInvalidCredentials(Boolean multiUser) {
-
-
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidCredentials: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidCredentials: Using authenticator: " + authenticator);
-      LogWriterUtils.getLogWriter()
-          .info("testInvalidCredentials: Using authinit: " + authInit);
-
-      // Start the servers
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = createServer1(extraProps, javaProps, authenticator,
-          locPort1, locString);
-      Integer port2 = ((Integer)server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, authenticator, extraProps,
-              javaProps )));
-
-      // Start first client with valid credentials
-      Properties credentials1 = gen.getValidCredentials(1);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidCredentials: For first client credentials: "
-              + credentials1 + " : " + javaProps1);
-      createClient1NoException(multiUser, authInit, port1, port2, credentials1,
-          javaProps1);
-
-      // Perform some put operations from client1
-      client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(2) ));
-
-      // Start second client with invalid credentials
-      // Trying to create the region on client2 should throw a security
-      // exception
-      Properties credentials2 = gen.getInvalidCredentials(1);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidCredentials: For second client credentials: "
-              + credentials2 + " : " + javaProps2);
-      client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2, javaProps2, port1, port2,
-              null, multiUser, new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) ));
-  }
-
-  public void testInvalidAuthInit() {
-    itestInvalidAuthInit(Boolean.FALSE);
-  }
-
-  public void itestInvalidAuthInit(Boolean multiUser) {
-
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthInit: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthInit: Using authenticator: " + authenticator);
-
-      // Start the server
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = createServer1(extraProps, javaProps, authenticator,
-          locPort1, locString);
-
-      Properties credentials = gen.getValidCredentials(1);
-      javaProps = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthInit: For first client credentials: " + credentials
-              + " : " + javaProps);
-      client1.invoke(ClientAuthenticationDUnitTest.class, "createCacheClient",
-          new Object[] { "com.gemstone.none", credentials, javaProps,
-            new Integer[] { port1 }, null, multiUser,
-            Integer.valueOf(SecurityTestUtil.AUTHREQ_EXCEPTION) });
-  }
-
-  protected Integer createServer1(Properties extraProps, Properties javaProps,
-      String authenticator, Integer locPort1, String locString) {
-    Integer port1 = ((Integer)server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, authenticator, extraProps,
-            javaProps )));
-    return port1;
-  }
-
-  public void testNoAuthInitWithCredentials() {
-    itestNoAuthInitWithCredentials(Boolean.FALSE);
-  }
-
-  public void itestNoAuthInitWithCredentials(Boolean multiUser) {
-
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-
-
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthInitWithCredentials: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthInitWithCredentials: Using authenticator: "
-              + authenticator);
-
-      // Start the servers
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = createServer1(extraProps, javaProps, authenticator,
-          locPort1, locString);
-      Integer port2 = ((Integer)server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, authenticator, extraProps,
-              javaProps )));
-
-      // Start the clients with valid credentials
-      Properties credentials1 = gen.getValidCredentials(1);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthInitWithCredentials: For first client credentials: "
-              + credentials1 + " : " + javaProps1);
-      Properties credentials2 = gen.getValidCredentials(2);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthInitWithCredentials: For second client credentials: "
-              + credentials2 + " : " + javaProps2);
-      client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, credentials1, javaProps1, port1, port2, null,
-          multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-      client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, credentials2, javaProps2, port1, port2, null,
-          multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-      client2.invoke(() -> SecurityTestUtil.closeCache());
-      
-
-      // Now also try with invalid credentials
-      Properties credentials3 = gen.getInvalidCredentials(5);
-      Properties javaProps3 = gen.getJavaProperties();
-      client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, credentials3, javaProps3, port1, port2, null,
-          multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-  }
-
-  public void testInvalidAuthenticator() {
-    itestInvalidAuthenticator(Boolean.FALSE);
-  }
-
-  public void itestInvalidAuthenticator(Boolean multiUser) {
-
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthenticator: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthenticator: Using authinit: " + authInit);
-
-      // Start the server with invalid authenticator
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = (Integer)server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, "com.gemstone.gemfire.none",
-              extraProps, javaProps ));
-
-      // Trying to create the region on client should throw a security exception
-      Properties credentials2 = gen.getValidCredentials(1);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthenticator: For first client credentials: "
-              + credentials2 + " : " + javaProps2);
-      client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2, javaProps2, port1, null,
-              new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) ));
-      client1.invoke(() -> SecurityTestUtil.closeCache());
-      
-
-      // Also test with invalid credentials
-      Properties credentials3 = gen.getInvalidCredentials(1);
-      Properties javaProps3 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testInvalidAuthenticator: For first client credentials: "
-              + credentials3 + " : " + javaProps3);
-      client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials3, javaProps3, port1, null,
-              new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) ));
-  }
-
-  public void testNoAuthenticatorWithCredentials() {
-    itestNoAuthenticatorWithCredentials(Boolean.FALSE);
-  }
-
-  public void itestNoAuthenticatorWithCredentials(Boolean multiUser) {
-
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthenticatorWithCredentials: Using scheme: "
-              + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthenticatorWithCredentials: Using authinit: " + authInit);
-
-      // Start the servers with no authenticator
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = (Integer)server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, null, extraProps, javaProps ));
-      Integer port2 = (Integer)server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, null, extraProps, javaProps ));
-
-      // Clients should connect successfully and work properly with
-      // valid/invalid credentials when none are required on the server side
-      Properties credentials1 = gen.getValidCredentials(3);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthenticatorWithCredentials: For first client credentials: "
-              + credentials1 + " : " + javaProps1);
-      Properties credentials2 = gen.getInvalidCredentials(5);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testNoAuthenticatorWithCredentials: For second client credentials: "
-              + credentials2 + " : " + javaProps2);
-      createClientsNoException(multiUser, authInit, port1, port2, credentials1,
-          javaProps1, credentials2, javaProps2);
-
-      // Perform some put operations from client1
-      client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(2) ));
-
-      // Verify that the puts succeeded
-      client2.invoke(() -> SecurityTestUtil.doGets( new Integer(2) ));
-  }
-
-  public void testCredentialsWithFailover() {
-    itestCredentialsWithFailover(Boolean.FALSE);
-  }
-
-  public void itestCredentialsWithFailover(Boolean multiUser) {
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsWithFailover: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsWithFailover: Using authenticator: " + authenticator);
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsWithFailover: Using authinit: " + authInit);
-
-      // Start the first server
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = (Integer)server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, authenticator, extraProps,
-              javaProps ));
-      // Get a port for second server but do not start it
-      // This forces the clients to connect to the first server
-      Integer port2 = new Integer(AvailablePort
-          .getRandomAvailablePort(AvailablePort.SOCKET));
-
-      // Start the clients with valid credentials
-      Properties credentials1 = gen.getValidCredentials(5);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsWithFailover: For first client credentials: "
-              + credentials1 + " : " + javaProps1);
-      Properties credentials2 = gen.getValidCredentials(6);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsWithFailover: For second client credentials: "
-              + credentials2 + " : " + javaProps2);
-      createClientsNoException(multiUser, authInit, port1, port2, credentials1,
-          javaProps1, credentials2, javaProps2);
-
-      // Perform some put operations from client1
-      client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(2) ));
-      // Verify that the puts succeeded
-      client2.invoke(() -> SecurityTestUtil.doGets( new Integer(2) ));
-
-      // start the second one and stop the first server to force a failover
-      server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, port2, authenticator, extraProps,
-              javaProps ));
-      server1.invoke(() -> SecurityTestUtil.closeCache());
-
-      // Perform some create/update operations from client1
-      client1.invoke(() -> SecurityTestUtil.doNPuts( new Integer(4) ));
-      // Verify that the creates/updates succeeded
-      client2.invoke(() -> SecurityTestUtil.doNGets( new Integer(4) ));
-
-      // Try to connect client2 with no credentials
-      // Verify that the creation of region throws security exception
-      if (gen.classCode().equals(ClassCode.SSL)) {
-        // For SSL the exception may not come since the server can close socket
-        // before handshake message is sent from client. However exception
-        // should come in any region operations.
-        client2
-            .invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, null, null, port1, port2, null, multiUser,
-                    new Integer(SecurityTestUtil.NOFORCE_AUTHREQ_EXCEPTION) ));
-        client2.invoke(() -> SecurityTestUtil.doPuts(
-            new Integer(2), new Integer(SecurityTestUtil.OTHER_EXCEPTION) ));
-      }
-      else {
-        client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, null, null, port1, port2,
-                null, multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-      }
-
-      // Now try to connect client1 with invalid credentials
-      // Verify that the creation of region throws security exception
-      Properties credentials3 = gen.getInvalidCredentials(7);
-      Properties javaProps3 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsWithFailover: For first client invalid credentials: "
-              + credentials3 + " : " + javaProps3);
-      client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials3, javaProps3, port1, port2,
-              null, multiUser, new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) ));
-
-      if (multiUser) {
-        client1.invoke(() -> SecurityTestUtil.doProxyCacheClose());
-        client2.invoke(() -> SecurityTestUtil.doProxyCacheClose());
-        client1.invoke(() -> SecurityTestUtil.doSimplePut("CacheClosedException"));
-        client2.invoke(() -> SecurityTestUtil.doSimpleGet("CacheClosedException"));
-      }
-  }
-
-  protected void createClientsNoException(Boolean multiUser, String authInit,
-      Integer port1, Integer port2, Properties credentials1,
-      Properties javaProps1, Properties credentials2, Properties javaProps2) {
-    createClient1NoException(multiUser, authInit, port1, port2, credentials1,
-        javaProps1);
-    client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2, javaProps2, port1, port2,
-            null, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) ));
-  }
-
-  public void testCredentialsForNotifications() {
-    itestCredentialsForNotifications(Boolean.FALSE);
-  }
-
-  public void itestCredentialsForNotifications(Boolean multiUser) {
-      CredentialGenerator gen = new DummyCredentialGenerator();
-      Properties extraProps = gen.getSystemProperties();
-      Properties javaProps = gen.getJavaProperties();
-      String authenticator = gen.getAuthenticator();
-      String authInit = gen.getAuthInit();
-
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsForNotifications: Using scheme: " + gen.classCode());
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsForNotifications: Using authenticator: "
-              + authenticator);
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsForNotifications: Using authinit: " + authInit);
-
-      // Start the first server
-      Integer locPort1 = SecurityTestUtil.getLocatorPort();
-      Integer locPort2 = SecurityTestUtil.getLocatorPort();
-      String locString = SecurityTestUtil.getLocatorString();
-      Integer port1 = (Integer)server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, authenticator, extraProps,
-              javaProps ));
-      // Get a port for second server but do not start it
-      // This forces the clients to connect to the first server
-      Integer port2 = new Integer(AvailablePort
-          .getRandomAvailablePort(AvailablePort.SOCKET));
-
-      // Start the clients with valid credentials
-      Properties credentials1 = gen.getValidCredentials(3);
-      Properties javaProps1 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsForNotifications: For first client credentials: "
-              + credentials1 + " : " + javaProps1);
-      Properties credentials2 = gen.getValidCredentials(4);
-      Properties javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsForNotifications: For second client credentials: "
-              + credentials2 + " : " + javaProps2);
-      createClient1NoException(multiUser, authInit, port1, port2, credentials1,
-          javaProps1);
-      // Set up zero forward connections to check notification handshake only
-      Object zeroConns = new Integer(0);
-      createClient2NoException(multiUser, authInit, port1, port2, credentials2,
-          javaProps2, zeroConns);
-
-      // Register interest on all keys on second client
-      client2
-          .invoke(() -> ClientAuthenticationDUnitTest.registerAllInterest());
-
-      // Perform some put operations from client1
-      client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(2) ));
-
-      // Verify that the puts succeeded
-      client2.invoke(() -> SecurityTestUtil.doLocalGets( new Integer(2) ));
-
-      // start the second one and stop the first server to force a failover
-      server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, port2, authenticator, extraProps,
-              javaProps ));
-      server1.invoke(() -> SecurityTestUtil.closeCache());
-
-      // Wait for failover to complete
-      Wait.pause(500);
-
-      // Perform some create/update operations from client1
-      client1.invoke(() -> SecurityTestUtil.doNPuts( new Integer(4) ));
-      // Verify that the creates/updates succeeded
-      client2.invoke(() -> SecurityTestUtil.doNLocalGets( new Integer(4) ));
-
-      // Try to connect client1 with no credentials
-      // Verify that the creation of region throws security exception
-      server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, port1, authenticator, extraProps,
-              javaProps ));
-      if (gen.classCode().equals(ClassCode.SSL)) {
-        // For SSL the exception may not come since the server can close socket
-        // before handshake message is sent from client. However exception
-        // should come in any region operations.
-        client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, null, null, port1, port2,
-                zeroConns, multiUser,
-                new Integer(SecurityTestUtil.NOFORCE_AUTHREQ_EXCEPTION) ));
-        client1.invoke(() -> SecurityTestUtil.doPuts(
-            new Integer(2), new Integer(SecurityTestUtil.OTHER_EXCEPTION) ));
-      }
-      else {
-        client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, null, null, port1, port2,
-                zeroConns, multiUser, new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-      }
-
-      // Now try to connect client2 with invalid credentials
-      // Verify that the creation of region throws security exception
-      credentials2 = gen.getInvalidCredentials(3);
-      javaProps2 = gen.getJavaProperties();
-      LogWriterUtils.getLogWriter().info(
-          "testCredentialsForNotifications: For second client invalid credentials: "
-              + credentials2 + " : " + javaProps2);
-      createClient2WithException(multiUser, authInit, port1, port2,
-          credentials2, javaProps2, zeroConns);
-
-      // Now try to connect client2 with invalid auth-init method
-      // Trying to create the region on client with valid credentials should
-      // throw a security exception
-      client2
-          .invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( "com.gemstone.none", credentials1, javaProps1,
-                  port1, port2, zeroConns, multiUser, 
-                  new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-
-      // Now start the servers with invalid authenticator method.
-      // Skip this test for a scheme which does not have an authInit in the
-      // first place (e.g. SSL) since that will fail with AuthReqEx before
-      // authenticator is even invoked.
-      if (authInit != null && authInit.length() > 0) {
-        server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, port1,
-                "com.gemstone.gemfire.none", extraProps, javaProps ));
-        server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, port2,
-                "com.gemstone.gemfire.none", extraProps, javaProps ));
-
-        createClient2WithException(multiUser, authInit, port1, port2,
-            credentials1, javaProps1, zeroConns);
-        createClient1WithException(multiUser, authInit, port1, port2,
-            credentials2, javaProps2, zeroConns);
-      }
-      else {
-        LogWriterUtils.getLogWriter().info(
-            "testCredentialsForNotifications: Skipping invalid authenticator for scheme ["
-                + gen.classCode() + "] which has no authInit");
-      }
-
-      // Try connection with null auth-init on clients.
-      // Skip this test for a scheme which does not have an authInit in the
-      // first place (e.g. SSL).
-      if (authInit != null && authInit.length() > 0) {
-        server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, port1,
-                authenticator, extraProps, javaProps ));
-        server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, port2,
-                authenticator, extraProps, javaProps ));
-        client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, credentials1, javaProps1,
-                port1, port2, null, multiUser,
-                new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-        createClient2AuthReqException(multiUser, port1, port2, credentials2,
-            javaProps2, zeroConns);
-
-        createClient2AuthReqException(multiUser, port1, port2, credentials2,
-            javaProps2, zeroConns);
-      }
-      else {
-        LogWriterUtils.getLogWriter().info(
-            "testCredentialsForNotifications: Skipping null authInit for scheme ["
-                + gen.classCode() + "] which has no authInit");
-      }
-
-      // Try connection with null authenticator on server and sending
-      // valid/invalid credentials.
-      // If the scheme does not have an authenticator in the first place (e.g.
-      // SSL) then skip it since this test is useless.
-      if (authenticator != null && authenticator.length() > 0) {
-        server1.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort1, locString, port1,
-                null, extraProps, javaProps ));
-        server2.invoke(() -> ClientAuthenticationDUnitTest.createCacheServer( locPort2, locString, port2,
-                null, extraProps, javaProps ));
-        createClient1NoException(multiUser, authInit, port1, port2,
-            credentials1, javaProps1);
-        createClient2NoException(multiUser, authInit, port1, port2,
-            credentials2, javaProps2, zeroConns);
-
-        // Register interest on all keys on second client
-        client2.invoke(() -> ClientAuthenticationDUnitTest.registerAllInterest());
-
-        // Perform some put operations from client1
-        client1.invoke(() -> SecurityTestUtil.doPuts( new Integer(4) ));
-
-        // Verify that the puts succeeded
-        client2.invoke(() -> SecurityTestUtil.doLocalGets( new Integer(4) ));
-
-        // Now also try with valid credentials on client2
-        createClient1NoException(multiUser, authInit, port1, port2,
-            credentials2, javaProps2);
-        createClient2NoException(multiUser, authInit, port1, port2,
-            credentials1, javaProps1, zeroConns);
-
-        // Register interest on all keys on second client
-        client2.invoke(() -> ClientAuthenticationDUnitTest.registerAllInterest());
-
-        // Perform some put operations from client1
-        client1.invoke(() -> SecurityTestUtil.doNPuts( new Integer(4) ));
-
-        // Verify that the puts succeeded
-        client2.invoke(() -> SecurityTestUtil.doNLocalGets( new Integer(4) ));
-      }
-      else {
-        LogWriterUtils.getLogWriter().info(
-            "testCredentialsForNotifications: Skipping scheme ["
-                + gen.classCode() + "] which has no authenticator");
-      }
-  }
-
-  protected void createClient1NoException(Boolean multiUser, String authInit,
-      Integer port1, Integer port2, Properties credentials2,
-      Properties javaProps2) {
-    client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2,
-            javaProps2, port1, port2, null, multiUser,
-            new Integer(SecurityTestUtil.NO_EXCEPTION) ));
-  }
-
-  protected void createClient2AuthReqException(Boolean multiUser, Integer port1,
-      Integer port2, Properties credentials2, Properties javaProps2,
-      Object zeroConns) {
-    client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( null, credentials2, javaProps2,
-            port1, port2, zeroConns, multiUser,
-            new Integer(SecurityTestUtil.AUTHREQ_EXCEPTION) ));
-  }
-
-  protected void createClient1WithException(Boolean multiUser, String authInit,
-      Integer port1, Integer port2, Properties credentials2,
-      Properties javaProps2, Object zeroConns) {
-    client1.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2,
-            javaProps2, port1, port2, zeroConns, multiUser,
-            new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) ));
-  }
-
-  protected void createClient2WithException(Boolean multiUser, String authInit,
-      Integer port1, Integer port2, Properties credentials2,
-      Properties javaProps2, Object zeroConns) {
-    client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2, javaProps2, port1, port2,
-            zeroConns, multiUser, new Integer(SecurityTestUtil.AUTHFAIL_EXCEPTION) ));
-  }
-
-  protected void createClient2NoException(Boolean multiUser, String authInit,
-      Integer port1, Integer port2, Properties credentials2,
-      Properties javaProps2, Object zeroConns) {
-    client2.invoke(() -> ClientAuthenticationDUnitTest.createCacheClient( authInit, credentials2, javaProps2, port1, port2,
-            zeroConns, multiUser, new Integer(SecurityTestUtil.NO_EXCEPTION) ));
-  }
-
-  //////////////////////////////////////////////////////////////////////////////
-  // Tests for MULTI_USER_MODE start here
-  //////////////////////////////////////////////////////////////////////////////
-
-  public void xtestValidCredentialsForMultipleUsers() {
-    itestValidCredentials(Boolean.TRUE);
+  @Test
+  public void testCredentialsForNotifications() throws Exception {
+    doTestCredentialsForNotifications(false);
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-  // Tests for MULTI_USER_MODE end here
-  //////////////////////////////////////////////////////////////////////////////
-  
-  @Override
-  public final void preTearDown() throws Exception {
-    // close the clients first
-    client1.invoke(() -> SecurityTestUtil.closeCache());
-    client2.invoke(() -> SecurityTestUtil.closeCache());
-    // then close the servers
-    server1.invoke(() -> SecurityTestUtil.closeCache());
-    server2.invoke(() -> SecurityTestUtil.closeCache());
+  @Ignore("Disabled for unknown reason")
+  @Test
+  public void testValidCredentialsForMultipleUsers() throws Exception {
+    doTestValidCredentials(true);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/22ca5ef8/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
index 138114a..b633865 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
@@ -1,6 +1,3 @@
-
-package com.gemstone.gemfire.security;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -9,9 +6,9 @@ package com.gemstone.gemfire.security;
  * 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
@@ -19,70 +16,59 @@ package com.gemstone.gemfire.security;
  * specific language governing permissions and limitations
  * under the License.
  */
+package com.gemstone.gemfire.security;
 
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 /**
  * this class contains test methods that used to be in its superclass but
  * that test started taking too long and caused dunit runs to hang
  */
-public class ClientAuthenticationPart2DUnitTest extends
-    ClientAuthenticationDUnitTest {
+@Category(DistributedTest.class)
+public class ClientAuthenticationPart2DUnitTest extends ClientAuthenticationTestCase {
 
-  /** constructor */
-  public ClientAuthenticationPart2DUnitTest(String name) {
-    super(name);
+  @Test
+  public void testNoCredentialsForMultipleUsers() throws Exception {
+    doTestNoCredentials(true);
   }
 
-  // override inherited tests so they aren't executed again
-  
-  @Override
-  public void testValidCredentials() {  }
-  @Override
-  public void testNoCredentials() {  }
-  @Override
-  public void testInvalidCredentials() {  }
-  @Override
-  public void testInvalidAuthInit() {  }
-  @Override
-  public void testNoAuthInitWithCredentials() {  }
-  @Override
-  public void testInvalidAuthenticator() {  }
-  @Override
-  public void testNoAuthenticatorWithCredentials() {  }
-  @Override
-  public void testCredentialsWithFailover() {  }
-  @Override
-  public void testCredentialsForNotifications() {  }
-  //@Override
-  public void testValidCredentialsForMultipleUsers() {  }
-
-
-  
-  
-  
-  public void testNoCredentialsForMultipleUsers() {
-    itestNoCredentials(Boolean.TRUE);
-  }
-  public void testInvalidCredentialsForMultipleUsers() {
-    itestInvalidCredentials(Boolean.TRUE);
+  @Test
+  public void testInvalidCredentialsForMultipleUsers() throws Exception {
+    doTestInvalidCredentials(true);
   }
-  public void testInvalidAuthInitForMultipleUsers() {
-    itestInvalidAuthInit(Boolean.TRUE);
-  }
-  public void testNoAuthInitWithCredentialsForMultipleUsers() {
-    itestNoAuthInitWithCredentials(Boolean.TRUE);
+
+  @Test
+  public void testInvalidAuthInitForMultipleUsers() throws Exception {
+    doTestInvalidAuthInit(true);
   }
-  public void testInvalidAuthenitcatorForMultipleUsers() {
-    itestInvalidAuthenticator(Boolean.TRUE);
+
+  @Test
+  public void testNoAuthInitWithCredentialsForMultipleUsers() throws Exception {
+    doTestNoAuthInitWithCredentials(true);
   }
-  public void testNoAuthenticatorWithCredentialsForMultipleUsers() {
-    itestNoAuthenticatorWithCredentials(Boolean.TRUE);
+
+  @Test
+  public void testInvalidAuthenitcatorForMultipleUsers() throws Exception {
+    doTestInvalidAuthenticator(true);
   }
-  public void disabled_testCredentialsWithFailoverForMultipleUsers() {
-    itestCredentialsWithFailover(Boolean.TRUE);
+
+  @Test
+  public void testNoAuthenticatorWithCredentialsForMultipleUsers() throws Exception {
+    doTestNoAuthenticatorWithCredentials(true);
   }
-  public void __testCredentialsForNotificationsForMultipleUsers() {
-    itestCredentialsForNotifications(Boolean.TRUE);
+
+  @Ignore("Disabled for unknown reason")
+  @Test
+  public void testCredentialsWithFailoverForMultipleUsers() throws Exception {
+    doTestCredentialsWithFailover(true);
   }
 
+  @Ignore("Disabled for unknown reason")
+  @Test
+  public void testCredentialsForNotificationsForMultipleUsers() throws Exception {
+    doTestCredentialsForNotifications(true);
+  }
 }