You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/02/03 03:32:46 UTC

[10/15] incubator-geode git commit: Refactor dunit

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
index 2038701..4474132 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
@@ -21,7 +21,9 @@ import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
+import org.apache.logging.log4j.Logger;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.SystemFailure;
@@ -51,6 +53,7 @@ import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
+import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
@@ -72,45 +75,15 @@ import junit.framework.TestCase;
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
 public abstract class DistributedTestCase extends TestCase implements java.io.Serializable {
+  private static final Logger logger = LogService.getLogger();
   
-  private static final LinkedHashSet<String> testHistory = new LinkedHashSet<String>();
+  private static final Set<String> testHistory = new LinkedHashSet<String>();
 
-  private static void setUpCreationStackGenerator() {
-    // the following is moved from InternalDistributedSystem to fix #51058
-    InternalDistributedSystem.TEST_CREATION_STACK_GENERATOR.set(
-    new CreationStackGenerator() {
-      @Override
-      public Throwable generateCreationStack(final DistributionConfig config) {
-        final StringBuilder sb = new StringBuilder();
-        final String[] validAttributeNames = config.getAttributeNames();
-        for (int i = 0; i < validAttributeNames.length; i++) {
-          final String attName = validAttributeNames[i];
-          final Object actualAtt = config.getAttributeObject(attName);
-          String actualAttStr = actualAtt.toString();
-          sb.append("  ");
-          sb.append(attName);
-          sb.append("=\"");
-          if (actualAtt.getClass().isArray()) {
-            actualAttStr = InternalDistributedSystem.arrayToString(actualAtt);
-          }
-          sb.append(actualAttStr);
-          sb.append("\"");
-          sb.append("\n");
-        }
-        return new Throwable("Creating distributed system with the following configuration:\n" + sb.toString());
-      }
-    });
-  }
-  
-  private static void tearDownCreationStackGenerator() {
-    InternalDistributedSystem.TEST_CREATION_STACK_GENERATOR.set(InternalDistributedSystem.DEFAULT_CREATION_STACK_GENERATOR);
-  }
-  
   /** This VM's connection to the distributed system */
   public static InternalDistributedSystem system;
   private static Class lastSystemCreatedInTest;
   private static Properties lastSystemProperties;
-  public static volatile String testName;
+  private static volatile String testMethodName;
   
   /** For formatting timing info */
   private static final DecimalFormat format = new DecimalFormat("###.###");
@@ -128,18 +101,14 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     DUnitLauncher.launchIfNeeded();
   }
 
-  protected Class getTestClass() {
-    Class clazz = getClass();
-    while (clazz.getDeclaringClass() != null) {
-      clazz = clazz.getDeclaringClass();
-    }
-    return clazz;
-  }
+  //---------------------------------------------------------------------------
+  // methods for tests
+  //---------------------------------------------------------------------------
   
-  public void setSystem(final Properties props, final DistributedSystem ds) {
+  public final void setSystem(final Properties props, final DistributedSystem ds) { // TODO: override getDistributedSystemProperties and then delete
     system = (InternalDistributedSystem)ds;
     lastSystemProperties = props;
-    lastSystemCreatedInTest = getTestClass();
+    lastSystemCreatedInTest = getClass(); // used to be getDeclaringClass()
   }
   
   /**
@@ -153,7 +122,7 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
    * see hydra.DistributedConnectionMgr#connect
    * @since 3.0
    */
-  public /*final*/ InternalDistributedSystem getSystem(final Properties props) {
+  public /*final*/ InternalDistributedSystem getSystem(final Properties props) { // TODO: make final
     // Setting the default disk store name is now done in setUp
     if (system == null) {
       system = InternalDistributedSystem.getAnyInstance();
@@ -161,9 +130,9 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     if (system == null || !system.isConnected()) {
       // Figure out our distributed system properties
       Properties p = DistributedTestSupport.getAllDistributedSystemProperties(props);
-      lastSystemCreatedInTest = getTestClass();
+      lastSystemCreatedInTest = getClass(); // used to be getDeclaringClass()
       if (logPerTest) {
-        String testMethod = getTestName();
+        String testMethod = getTestMethodName();
         String testName = lastSystemCreatedInTest.getName() + '-' + testMethod;
         String oldLogFile = p.getProperty(DistributionConfig.LOG_FILE_NAME);
         p.put(DistributionConfig.LOG_FILE_NAME, 
@@ -176,7 +145,7 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
       lastSystemProperties = p;
     } else {
       boolean needNewSystem = false;
-      if(!getTestClass().equals(lastSystemCreatedInTest)) {
+      if(!getClass().equals(lastSystemCreatedInTest)) { // used to be getDeclaringClass()
         Properties newProps = DistributedTestSupport.getAllDistributedSystemProperties(props);
         needNewSystem = !newProps.equals(lastSystemProperties);
         if(needNewSystem) {
@@ -212,11 +181,6 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     return system;
   }
 
-  private String getDefaultDiskStoreName() {
-    String vmid = System.getProperty("vmid");
-    return "DiskStore-"  + vmid + "-"+ getTestClass().getCanonicalName() + "." + getTestName();
-  }
-
   /**
    * Returns this VM's connection to the distributed system.  If
    * necessary, the connection will be lazily created using the
@@ -228,7 +192,7 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
    * @since 3.0
    */
   public final InternalDistributedSystem getSystem() {
-    return getSystem(this.getDistributedSystemProperties());
+    return getSystem(getDistributedSystemProperties());
   }
 
   /**
@@ -238,7 +202,7 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
    * @since 6.5
    */
   public final InternalDistributedSystem getLonerSystem() {
-    Properties props = this.getDistributedSystemProperties();
+    Properties props = getDistributedSystemProperties();
     props.put(DistributionConfig.MCAST_PORT_NAME, "0");
     props.put(DistributionConfig.LOCATORS_NAME, "");
     return getSystem(props);
@@ -250,7 +214,7 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
    * Added specifically to test scenario of defect #47181.
    */
   public final InternalDistributedSystem getLonerSystemWithEnforceUniqueHost() {
-    Properties props = this.getDistributedSystemProperties();
+    Properties props = getDistributedSystemProperties();
     props.put(DistributionConfig.MCAST_PORT_NAME, "0");
     props.put(DistributionConfig.LOCATORS_NAME, "");
     props.put(DistributionConfig.ENFORCE_UNIQUE_HOST_NAME, "true");
@@ -279,219 +243,286 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     return new Properties();
   }
 
+  public static void disconnectAllFromDS() {
+    disconnectFromDS();
+    Invoke.invokeInEveryVM(()->disconnectFromDS());
+  }
+
+  /**
+   * Disconnects this VM from the distributed system
+   */
+  public static void disconnectFromDS() {
+    setTestMethodName(null);
+    GemFireCacheImpl.testCacheXml = null;
+    if (system != null) {
+      system.disconnect();
+      system = null;
+    }
+    
+    for (;;) {
+      DistributedSystem ds = InternalDistributedSystem.getConnectedInstance();
+      if (ds == null) {
+        break;
+      }
+      try {
+        ds.disconnect();
+      } catch (Exception e) {
+        // ignore
+      }
+    }
+    
+    AdminDistributedSystemImpl ads = AdminDistributedSystemImpl.getConnectedInstance();
+    if (ads != null) {// && ads.isConnected()) {
+      ads.disconnect();
+    }
+  }
+
+  //---------------------------------------------------------------------------
+  // name methods
+  //---------------------------------------------------------------------------
+  
+  public static String getTestMethodName() {
+    return testMethodName;
+  }
+
+  public static void setTestMethodName(final String testMethodName) { // TODO: delete
+    DistributedTestCase.testMethodName = testMethodName;
+  }
+  
+  /**
+   * Returns a unique name for this test method.  It is based on the
+   * name of the class as well as the name of the method.
+   */
+  public String getUniqueName() {
+    return getClass().getSimpleName() + "_" + getName();
+  }
+
+  //---------------------------------------------------------------------------
+  // setup methods
+  //---------------------------------------------------------------------------
+  
   /**
-   * Sets up the test (noop).
+   * Sets up the DistributedTestCase.
+   * <p>
+   * Do not override this method. Override {@link #preSetUp()} with work that
+   * needs to occur before setUp() or override {@link #postSetUp()} with work
+   * that needs to occur after setUp().
    */
   @Override
   public void setUp() throws Exception {
+    preSetUp();
+    setUpDistributedTestCase();
+    postSetUp();
+  }
+  
+  /**
+   * Sets up DistributedTest in controller and remote VMs. This includes the
+   * defining the test name, setting the default disk store name, logging the 
+   * test history, and capturing a creation stack for detecting the source of
+   * incompatible DistributedSystem connections.
+   * <p>
+   * Do not override this method.
+   */
+  private final void setUpDistributedTestCase() {
+    final String testMethodName = getName();
     logTestHistory();
-    setUpCreationStackGenerator();
-    testName = getName();
-    System.setProperty(HoplogConfig.ALLOW_LOCAL_HDFS_PROP, "true");
     
-    if (testName != null) {
-      GemFireCacheImpl.setDefaultDiskStoreName(getDefaultDiskStoreName());
-      String baseDefaultDiskStoreName = getTestClass().getCanonicalName() + "." + getTestName();
-      for (int h = 0; h < Host.getHostCount(); h++) {
-        Host host = Host.getHost(h);
-        for (int v = 0; v < host.getVMCount(); v++) {
-          VM vm = host.getVM(v);
-          String vmDefaultDiskStoreName = "DiskStore-" + h + "-" + v + "-" + baseDefaultDiskStoreName;
-          vm.invoke(DistributedTestCase.class, "perVMSetUp", new Object[] {testName, vmDefaultDiskStoreName});
-        }
+    setUpVM(testMethodName, getDefaultDiskStoreName(0, -1));
+    
+    for (int hostIndex = 0; hostIndex < Host.getHostCount(); hostIndex++) {
+      Host host = Host.getHost(hostIndex);
+      for (int vmIndex = 0; vmIndex < host.getVMCount(); vmIndex++) {
+        final String vmDefaultDiskStoreName = getDefaultDiskStoreName(hostIndex, vmIndex);
+        host.getVM(vmIndex).invoke(()->setUpVM(testMethodName, vmDefaultDiskStoreName));
       }
     }
-    System.out.println("\n\n[setup] START TEST " + getClass().getSimpleName()+"."+testName+"\n\n");
+    
+    logTestStart();
   }
 
   /**
-   * Write a message to the log about what tests have ran previously. This
-   * makes it easier to figure out if a previous test may have caused problems
+   * <code>preSetUp()</code> is invoked before {@link #setUpDistributedTestCase()}.
+   * <p>
+   * Override this as needed. Default implementation is empty.
    */
-  private void logTestHistory() {
-    String classname = getClass().getSimpleName();
-    testHistory.add(classname);
-    System.out.println("Previously run tests: " + testHistory);
+  protected void preSetUp() throws Exception {
   }
-
-  public static void perVMSetUp(final String name, final String defaultDiskStoreName) {
-    setTestName(name);
-    GemFireCacheImpl.setDefaultDiskStoreName(defaultDiskStoreName);
-    System.setProperty(HoplogConfig.ALLOW_LOCAL_HDFS_PROP, "true");    
+  
+  /**
+   * <code>postSetUp()</code> is invoked after {@link #setUpDistributedTestCase()}.
+   * <p>
+   * Override this as needed. Default implementation is empty.
+   */
+  protected void postSetUp() throws Exception {
   }
   
-  private static void setTestName(final String name) {
-    testName = name;
+  private String getDefaultDiskStoreName(final int hostIndex, final int vmIndex) {
+    return "DiskStore-" + String.valueOf(hostIndex) + "-" + String.valueOf(vmIndex) + "-" + getClass().getCanonicalName() + "." + getTestMethodName(); // used to be getDeclaringClass()
   }
   
-  public static String getTestName() {
-    return testName;
+  private static void setUpVM(final String testMethodName, final String defaultDiskStoreName) {
+    setTestMethodName(testMethodName);
+    GemFireCacheImpl.setDefaultDiskStoreName(defaultDiskStoreName);
+    System.setProperty(HoplogConfig.ALLOW_LOCAL_HDFS_PROP, "true");    
+    setUpCreationStackGenerator();
   }
 
+  private void logTestStart() {
+    System.out.println("\n\n[setup] START TEST " + getClass().getSimpleName()+"."+testMethodName+"\n\n");
+  }
+  
+  private static void setUpCreationStackGenerator() {
+    // the following is moved from InternalDistributedSystem to fix #51058
+    InternalDistributedSystem.TEST_CREATION_STACK_GENERATOR.set(
+    new CreationStackGenerator() {
+      @Override
+      public Throwable generateCreationStack(final DistributionConfig config) {
+        final StringBuilder sb = new StringBuilder();
+        final String[] validAttributeNames = config.getAttributeNames();
+        for (int i = 0; i < validAttributeNames.length; i++) {
+          final String attName = validAttributeNames[i];
+          final Object actualAtt = config.getAttributeObject(attName);
+          String actualAttStr = actualAtt.toString();
+          sb.append("  ");
+          sb.append(attName);
+          sb.append("=\"");
+          if (actualAtt.getClass().isArray()) {
+            actualAttStr = InternalDistributedSystem.arrayToString(actualAtt);
+          }
+          sb.append(actualAttStr);
+          sb.append("\"");
+          sb.append("\n");
+        }
+        return new Throwable("Creating distributed system with the following configuration:\n" + sb.toString());
+      }
+    });
+  }
+  
   /**
-   * For logPerTest to work, we have to disconnect from the DS, but all
-   * subclasses do not call super.tearDown(). To prevent this scenario
-   * this method has been declared final. Subclasses must now override
-   * {@link #tearDown2()} instead.
-   * @throws Exception
+   * Write a message to the log about what tests have ran previously. This
+   * makes it easier to figure out if a previous test may have caused problems
    */
-  @Override
-  public final void tearDown() throws Exception {
-    tearDownCreationStackGenerator();
-    tearDown2();
-    realTearDown();
-    tearDownAfter();
+  private void logTestHistory() {
+    String classname = getClass().getSimpleName();
+    testHistory.add(classname);
+    System.out.println("Previously run tests: " + testHistory);
   }
 
+  //---------------------------------------------------------------------------
+  // teardown methods
+  //---------------------------------------------------------------------------
+
   /**
-   * Tears down the test. This method is called by the final {@link #tearDown()} method and should be overridden to
-   * perform actual test cleanup and release resources used by the test.  The tasks executed by this method are
-   * performed before the DUnit test framework using Hydra cleans up the client VMs.
-   * <p/>
-   * @throws Exception if the tear down process and test cleanup fails.
-   * @see #tearDown
-   * @see #tearDownAfter()
+   * Tears down the DistributedTestCase.
+   * <p>
+   * Do not override this method. Override {@link #preTearDown()} with work that
+   * needs to occur before tearDown() or override {@link #postTearDown()} with work
+   * that needs to occur after tearDown().
    */
-  // TODO rename this method to tearDownBefore and change the access modifier to protected!
-  public void tearDown2() throws Exception {
+  @Override
+  public final void tearDown() throws Exception {
+    preTearDown();
+    tearDownDistributedTestCase();
+    postTearDown();
   }
 
-  protected void realTearDown() throws Exception {
+  private final void tearDownDistributedTestCase() throws Exception {
+    Invoke.invokeInEveryVM(()->tearDownCreationStackGenerator());
     if (logPerTest) {
       disconnectFromDS();
-      Invoke.invokeInEveryVM(DistributedTestCase.class, "disconnectFromDS");
+      Invoke.invokeInEveryVM(()->disconnectFromDS());
     }
     cleanupAllVms();
   }
   
   /**
-   * Tears down the test.  Performs additional tear down tasks after the DUnit tests framework using Hydra cleans up
-   * the client VMs.  This method is called by the final {@link #tearDown()} method and should be overridden to perform
-   * post tear down activities.
-   * <p/>
-   * @throws Exception if the test tear down process fails.
-   * @see #tearDown()
-   * @see #tearDown2()
+   * <code>preTearDown()</code> is invoked before {@link #tearDownDistributedTestCase()}.
+   * <p>
+   * Override this as needed. Default implementation is empty.
    */
-  protected void tearDownAfter() throws Exception {
+  protected void preTearDown() throws Exception {
   }
-
-  public static void cleanupAllVms() {
-    cleanupThisVM();
-    Invoke.invokeInEveryVM(()->cleanupThisVM());
-    Invoke.invokeInLocator(new SerializableRunnable() {
-      public void run() {
-        DistributionMessageObserver.setInstance(null);
-        DistributedTestSupport.unregisterInstantiatorsInThisVM();
-      }
+  
+  /**
+   * <code>postTearDown()</code> is invoked after {@link #tearDownDistributedTestCase()}.
+   * <p>
+   * Override this as needed. Default implementation is empty.
+   */
+  protected void postTearDown() throws Exception {
+  }
+  
+  public static void cleanupAllVms() { // TODO: make private
+    tearDownVM();
+    Invoke.invokeInEveryVM(()->tearDownVM());
+    Invoke.invokeInLocator(()->{
+      DistributionMessageObserver.setInstance(null);
+      DistributedTestSupport.unregisterInstantiatorsInThisVM();
     });
     DUnitLauncher.closeAndCheckForSuspects();
   }
 
-  private static void cleanupThisVM() {
+  private static void tearDownVM() {
     closeCache();
-    
-    SocketCreator.resolve_dns = true;
+
+    // keep alphabetized to detect duplicate lines
     CacheCreation.clearThreadLocals();
-    System.getProperties().remove("gemfire.log-level");
-    System.getProperties().remove("jgroups.resolve_dns");
-    InitialImageOperation.slowImageProcessing = 0;
-    DistributionMessageObserver.setInstance(null);
-    QueryTestUtils.setCache(null);
     CacheServerTestUtil.clearCacheReference();
-    RegionTestCase.preSnapshotRegion = null;
-    GlobalLockingDUnitTest.region_testBug32356 = null;
-    LogWrapper.close();
     ClientProxyMembershipID.system = null;
-    MultiVMRegionTestCase.CCRegion = null;
-    InternalClientMembership.unregisterAllListeners();
-    ClientStatsManager.cleanupForTests();
     ClientServerTestCase.AUTO_LOAD_BALANCE = false;
+    ClientStatsManager.cleanupForTests();
+    DiskStoreObserver.setInstance(null);
     DistributedTestSupport.unregisterInstantiatorsInThisVM();
     DistributionMessageObserver.setInstance(null);
+    GlobalLockingDUnitTest.region_testBug32356 = null;
+    InitialImageOperation.slowImageProcessing = 0;
+    InternalClientMembership.unregisterAllListeners();
+    LogWrapper.close();
+    MultiVMRegionTestCase.CCRegion = null;
     QueryObserverHolder.reset();
-    DiskStoreObserver.setInstance(null);
-    System.getProperties().remove("gemfire.log-level");
-    System.getProperties().remove("jgroups.resolve_dns");
+    QueryTestUtils.setCache(null);
+    RegionTestCase.preSnapshotRegion = null;
+    SocketCreator.resolve_dns = true;
+
+    // clear system properties -- keep alphabetized
+    System.clearProperty("gemfire.log-level");
+    System.clearProperty(HoplogConfig.ALLOW_LOCAL_HDFS_PROP);    
+    System.clearProperty("jgroups.resolve_dns");
     
     if (InternalDistributedSystem.systemAttemptingReconnect != null) {
       InternalDistributedSystem.systemAttemptingReconnect.stopReconnecting();
     }
+    
     IgnoredException.removeAllExpectedExceptions();
   }
 
   private static void closeCache() {
     GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-    if(cache != null && !cache.isClosed()) {
+    if (cache != null && !cache.isClosed()) {
       destroyRegions(cache);
       cache.close();
     }
   }
   
-  protected static final void destroyRegions(final Cache cache) {
+  protected static final void destroyRegions(final Cache cache) { // TODO: make private
     if (cache != null && !cache.isClosed()) {
-      //try to destroy the root regions first so that
-      //we clean up any persistent files.
+      // try to destroy the root regions first so that we clean up any persistent files.
       for (Iterator itr = cache.rootRegions().iterator(); itr.hasNext();) {
         Region root = (Region)itr.next();
-        //for colocated regions you can't locally destroy a partitioned
-        //region.
+        String regionFullPath = root == null ? null : root.getFullPath();
+        // for colocated regions you can't locally destroy a partitioned region.
         if(root.isDestroyed() || root instanceof HARegion || root instanceof PartitionedRegion) {
           continue;
         }
         try {
           root.localDestroyRegion("teardown");
-        }
-        catch (VirtualMachineError e) {
-          SystemFailure.initiateFailure(e);
-          throw e;
-        }
-        catch (Throwable t) {
-          LogWriterSupport.getLogWriter().error(t);
+        } catch (Throwable t) {
+          logger.error("Failure during tearDown destroyRegions for " + regionFullPath, t);
         }
       }
     }
   }
   
-  public static void disconnectAllFromDS() {
-    disconnectFromDS();
-    Invoke.invokeInEveryVM(DistributedTestCase.class, "disconnectFromDS");
-  }
-
-  /**
-   * Disconnects this VM from the distributed system
-   */
-  public static void disconnectFromDS() {
-    testName = null;
-    GemFireCacheImpl.testCacheXml = null;
-    if (system != null) {
-      system.disconnect();
-      system = null;
-    }
-    
-    for (;;) {
-      DistributedSystem ds = InternalDistributedSystem.getConnectedInstance();
-      if (ds == null) {
-        break;
-      }
-      try {
-        ds.disconnect();
-      } catch (Exception e) {
-        // ignore
-      }
-    }
-    
-    AdminDistributedSystemImpl ads = AdminDistributedSystemImpl.getConnectedInstance();
-    if (ads != null) {// && ads.isConnected()) {
-      ads.disconnect();
-    }
-  }
-
-  /**
-   * Returns a unique name for this test method.  It is based on the
-   * name of the class as well as the name of the method.
-   */
-  public String getUniqueName() {
-    return getClass().getSimpleName() + "_" + getName();
+  private static void tearDownCreationStackGenerator() {
+    InternalDistributedSystem.TEST_CREATION_STACK_GENERATOR.set(InternalDistributedSystem.DEFAULT_CREATION_STACK_GENERATOR);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/Invoke.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/Invoke.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/Invoke.java
index cc6d91f..edb742b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/Invoke.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/Invoke.java
@@ -27,6 +27,8 @@ public class Invoke {
   /**
    * Invokes a <code>SerializableRunnable</code> in every VM that
    * DUnit knows about.
+   * <p>
+   * Apparently this does NOT include the controller VM.
    *
    * @see VM#invoke(Runnable)
    */
@@ -92,7 +94,7 @@ public class Invoke {
     return ret;
   }
 
-  public static void invokeInLocator(SerializableRunnable work) {
+  public static void invokeInLocator(SerializableRunnableIF work) {
     Host.getLocator().invoke(work);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
new file mode 100755
index 0000000..2138d78
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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 com.gemstone.gemfire.test.dunit.tests;
+
+import static org.assertj.core.api.Assertions.*;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@SuppressWarnings("serial")
+@Category(DistributedTest.class)
+public class GetDefaultDiskStoreNameDUnitTest extends DistributedTestCase {
+
+  public GetDefaultDiskStoreNameDUnitTest(final String name) {
+    super(name);
+  }
+
+  public void testGetTestMethodName() {
+    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodName");
+    assertGetDefaultDiskStoreName(expected);
+  }
+  
+  public void testGetTestMethodNameChanges() {
+    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameChanges");
+    assertGetDefaultDiskStoreName(expected);
+  }
+  
+  public void testGetTestMethodNameInAllVMs() {
+    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameInAllVMs");
+    assertGetDefaultDiskStoreName(expected);
+    
+    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
+      String expectedInVM = createDefaultDiskStoreName(0, vmIndex, "testGetTestMethodNameInAllVMs");
+      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetDefaultDiskStoreName(expectedInVM));
+    }
+  }
+  
+  private void assertGetDefaultDiskStoreName(final String expected) {
+    assertThat(getDefaultDiskStoreName()).isEqualTo(expected);
+  }
+  
+  private String createDefaultDiskStoreName(final int hostIndex, final int vmIndex, final String testMethodName) {
+    return "DiskStore-" + hostIndex + "-" + vmIndex + "-" + getClass().getCanonicalName() + "." + testMethodName;
+  }
+  
+  private String getDefaultDiskStoreName() {
+    return GemFireCacheImpl.DEFAULT_DS_NAME; // TODO: not thread safe
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java
new file mode 100755
index 0000000..9bad472
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.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 com.gemstone.gemfire.test.dunit.tests;
+
+import static org.assertj.core.api.Assertions.*;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@SuppressWarnings("serial")
+@Category(DistributedTest.class)
+public class GetTestMethodNameDUnitTest extends DistributedTestCase {
+
+  public GetTestMethodNameDUnitTest(final String name) {
+    super(name);
+  }
+
+  public void testGetTestMethodName() {
+    assertGetTestMethodName("testGetTestMethodName");
+  }
+  
+  public void testGetTestMethodNameChanges() {
+    assertGetTestMethodName("testGetTestMethodNameChanges");
+  }
+  
+  public void testGetTestMethodNameInAllVMs() {
+    assertGetTestMethodName("testGetTestMethodNameInAllVMs");
+    
+    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
+      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetTestMethodName("testGetTestMethodNameInAllVMs"));
+    }
+  }
+  
+  private void assertGetTestMethodName(final String expected) {
+    assertThat(getTestMethodName()).isEqualTo(expected);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/AssertJExampleJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/AssertJExampleJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/AssertJExampleJUnitTest.java
deleted file mode 100755
index ac0a373..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/AssertJExampleJUnitTest.java
+++ /dev/null
@@ -1,135 +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 com.gemstone.gemfire.test.examples;
-
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class AssertJExampleJUnitTest {
-
-  private Character aragorn;
-  private Character boromir;
-  private Character elrond;
-  private Character frodo;
-  private Character galadriel;
-  private Character gandalf;
-  private Character gimli;
-  private Character legolas;
-  private Character merry;
-  private Character pippin;
-  private Character sauron;
-  private Character sam;
-  
-  private Ring narya;
-  private Ring nenya;
-  private Ring oneRing;
-  private Ring vilya;
-  
-  private Set<Character> fellowshipOfTheRing;
-  private Map<Ring, Character> ringBearers;
-  
-  @Before
-  public void setUp() {
-    this.aragorn = new Character("Aragorn");
-    this.boromir = new Character("Boromir");
-    this.elrond = new Character("Elrond");
-    this.frodo = new Character("Frodo");
-    this.galadriel = new Character("Galadriel");
-    this.gandalf = new Character("Gandalf");
-    this.gimli = new Character("Gimli");
-    this.legolas = new Character("Legolas");
-    this.merry = new Character("Merry");
-    this.pippin = new Character("Pippin");
-    this.sauron = new Character("Sauron");
-    this.sam = new Character("Sam");
-    
-    this.narya = new Ring();
-    this.nenya = new Ring();
-    this.oneRing = new Ring();
-    this.vilya = new Ring();
-    
-    this.fellowshipOfTheRing = new HashSet<Character>();
-    this.fellowshipOfTheRing.add(this.aragorn);
-    this.fellowshipOfTheRing.add(this.boromir);
-    this.fellowshipOfTheRing.add(this.frodo);
-    this.fellowshipOfTheRing.add(this.gandalf);
-    this.fellowshipOfTheRing.add(this.gimli);
-    this.fellowshipOfTheRing.add(this.legolas);
-    this.fellowshipOfTheRing.add(this.merry);
-    this.fellowshipOfTheRing.add(this.pippin);
-    this.fellowshipOfTheRing.add(this.sam);
-
-    this.ringBearers = new HashMap<Ring, Character>();
-    this.ringBearers.put(this.oneRing, this.frodo);
-    this.ringBearers.put(this.nenya, this.galadriel);
-    this.ringBearers.put(this.narya, this.gandalf);
-    this.ringBearers.put(this.vilya, this.elrond);
-  }
-  
-  @Test
-  public void exampleShouldPass() {
-    // common assertions
-    assertThat(frodo.getName()).isEqualTo("Frodo");
-    assertThat(frodo).isNotEqualTo(sauron)
-                     .isIn(fellowshipOfTheRing);
-
-    // String specific assertions
-    assertThat(frodo.getName()).startsWith("Fro")
-                               .endsWith("do")
-                               .isEqualToIgnoringCase("frodo");
-
-    // collection specific assertions
-    assertThat(fellowshipOfTheRing).hasSize(9)
-                                   .contains(frodo, sam)
-                                   .doesNotContain(sauron);
-
-    // using extracting magical feature to check fellowshipOfTheRing characters name :)
-    assertThat(fellowshipOfTheRing).extracting("name").contains("Boromir", "Gandalf", "Frodo", "Legolas")
-                                                      .doesNotContain("Sauron", "Elrond");
-
-    // map specific assertions, ringBearers initialized with the elves rings and the one ring bearers.
-    assertThat(ringBearers).hasSize(4)
-                           .contains(entry(oneRing, frodo), entry(nenya, galadriel))
-                           .doesNotContainEntry(oneRing, aragorn);  
-  }
-  
-  protected static class Character {
-    private final String name;
-    public Character(final String name) {
-      this.name = name;
-    }
-    public String getName() {
-      return this.name;
-    }
-  }
-  
-  protected static class Ring {
-    public Ring() {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/CatchExceptionExampleJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/CatchExceptionExampleJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/CatchExceptionExampleJUnitTest.java
deleted file mode 100755
index 264b563..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/CatchExceptionExampleJUnitTest.java
+++ /dev/null
@@ -1,115 +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 com.gemstone.gemfire.test.examples;
-
-import static com.googlecode.catchexception.CatchException.*;
-import static com.googlecode.catchexception.apis.BDDCatchException.when;
-import static com.googlecode.catchexception.apis.CatchExceptionHamcrestMatchers.*;
-import static org.assertj.core.api.BDDAssertions.*;
-import static org.hamcrest.Matchers.*;
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- * Simple unit tests exercising Catch-Exception with AssertJ, Hamcrest and JUnit.
- */
-@Category(UnitTest.class)
-public class CatchExceptionExampleJUnitTest {
-
-  @Test
-  public void catchExceptionShouldCatchException() {
-    List<?> myList = new ArrayList<Object>();
-
-    // when: we try to get the first element of the list
-    // then: catch the exception if any is thrown
-    catchException(myList).get(1);
-    
-    // then: we expect an IndexOutOfBoundsException
-    assertThat(caughtException(), is(instanceOf(IndexOutOfBoundsException.class)));
-  }
-  
-  @Test
-  public void verifyExceptionShouldCatchException() {
-    List<?> myList = new ArrayList<Object>();
-
-    // when: we try to get the first element of the list
-    // then: catch the exception if any is thrown
-    // then: we expect an IndexOutOfBoundsException
-    verifyException(myList, IndexOutOfBoundsException.class).get(1);
-  }
-  
-  @Test
-  public void whenShouldCatchExceptionAndUseAssertJAssertion() {
-    // given: an empty list
-    List<?> myList = new ArrayList<Object>();
-
-    // when: we try to get the first element of the list
-    when(myList).get(1);
-
-    // then: we expect an IndexOutOfBoundsException
-    then((Exception)caughtException())
-            .isInstanceOf(IndexOutOfBoundsException.class)
-            .hasMessage("Index: 1, Size: 0")
-            .hasNoCause();
-  }
-  
-  @Test
-  public void catchExceptionShouldCatchExceptionAndUseHamcrestAssertion() {
-    // given: an empty list
-    List<?> myList = new ArrayList<Object>();
-
-    // when: we try to get the first element of the list
-    catchException(myList).get(1);
-
-    // then: we expect an IndexOutOfBoundsException with message "Index: 1, Size: 0"
-    assertThat(caughtException(),
-      allOf(
-        instanceOf(IndexOutOfBoundsException.class),
-        hasMessage("Index: 1, Size: 0"),
-        hasNoCause()
-      )
-    );
-  }
-  
-  @Test
-  public void shouldCatchFromThrowException() throws Exception {
-    String message = "error message";
-    
-    catchException(this).throwException(message);
-    
-    assertThat(caughtException(), is(instanceOf(Exception.class)));
-  }
-  
-  @Test
-  public void shouldVerifyFromThrowException() throws Exception {
-    String message = "error message";
-
-    verifyException(this).throwException(message);
-  }
-  
-  // fails if private
-  protected void throwException(final String message) throws Exception {
-    throw new Exception(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/JUnitParamsExampleJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/JUnitParamsExampleJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/JUnitParamsExampleJUnitTest.java
deleted file mode 100755
index 3e9edb0..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/examples/JUnitParamsExampleJUnitTest.java
+++ /dev/null
@@ -1,52 +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 com.gemstone.gemfire.test.examples;
-
-import static org.hamcrest.Matchers.*;
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-
-@Category(UnitTest.class)
-@RunWith(JUnitParamsRunner.class)
-public class JUnitParamsExampleJUnitTest {
-  @Test
-  @Parameters({"17, false", 
-               "22, true" })
-  public void personIsAdult(int age, boolean valid) throws Exception {
-    assertThat(true, is(true));
-    assertThat(new Person(age).isAdult(), is(valid));
-  }
-  
-  protected static class Person {
-    private static final int MIN_AGE_OF_ADULT = 18;
-    private final int age;
-    public Person(final int age) {
-      this.age = age;
-    }
-    public Boolean isAdult() {
-      return this.age >= MIN_AGE_OF_ADULT;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataOptimizedExecuteDUnitTest.java
index 6b97fc9..e185af1 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataOptimizedExecuteDUnitTest.java
@@ -45,12 +45,11 @@ public class CqDataOptimizedExecuteDUnitTest extends CqDataDUnitTest{
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataUsingPoolOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataUsingPoolOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataUsingPoolOptimizedExecuteDUnitTest.java
index 58f4dd6..47b9a89 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataUsingPoolOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqDataUsingPoolOptimizedExecuteDUnitTest.java
@@ -44,12 +44,11 @@ public class CqDataUsingPoolOptimizedExecuteDUnitTest extends CqDataUsingPoolDUn
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryOptimizedExecuteDUnitTest.java
index d6481eb..32b9f94 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryOptimizedExecuteDUnitTest.java
@@ -52,14 +52,13 @@ public class CqQueryOptimizedExecuteDUnitTest extends CqQueryDUnitTest{
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
         CqServiceProvider.MAINTAIN_KEYS = true;
       }
     });
-    super.tearDown2();
   }
   
   public void testCqExecuteWithoutQueryExecution() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolOptimizedExecuteDUnitTest.java
index 4febfd3..cf3dc95 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqQueryUsingPoolOptimizedExecuteDUnitTest.java
@@ -40,12 +40,11 @@ public class CqQueryUsingPoolOptimizedExecuteDUnitTest extends CqQueryUsingPoolD
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqResultSetUsingPoolOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqResultSetUsingPoolOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqResultSetUsingPoolOptimizedExecuteDUnitTest.java
index 88fad71..430e18d 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqResultSetUsingPoolOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqResultSetUsingPoolOptimizedExecuteDUnitTest.java
@@ -55,13 +55,12 @@ public class CqResultSetUsingPoolOptimizedExecuteDUnitTest extends CqResultSetUs
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsOptimizedExecuteDUnitTest.java
index e02e3b4..df0b950 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsOptimizedExecuteDUnitTest.java
@@ -40,12 +40,11 @@ public class CqStatsOptimizedExecuteDUnitTest extends CqStatsDUnitTest{
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolOptimizedExecuteDUnitTest.java
index fc1fc04..a959024 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStatsUsingPoolOptimizedExecuteDUnitTest.java
@@ -40,12 +40,11 @@ public class CqStatsUsingPoolOptimizedExecuteDUnitTest extends CqStatsUsingPoolD
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryOptimizedExecuteDUnitTest.java
index 2990503..97cfd7c 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PartitionedRegionCqQueryOptimizedExecuteDUnitTest.java
@@ -46,14 +46,13 @@ public class PartitionedRegionCqQueryOptimizedExecuteDUnitTest extends Partition
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
         CqServiceProvider.MAINTAIN_KEYS = true;
       }
     });
-    super.tearDown2();
   }
   
   public void testCqExecuteWithoutQueryExecution() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PrCqUsingPoolOptimizedExecuteDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PrCqUsingPoolOptimizedExecuteDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PrCqUsingPoolOptimizedExecuteDUnitTest.java
index 9ebc396..d71a7be 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PrCqUsingPoolOptimizedExecuteDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/PrCqUsingPoolOptimizedExecuteDUnitTest.java
@@ -40,12 +40,11 @@ public class PrCqUsingPoolOptimizedExecuteDUnitTest extends PrCqUsingPoolDUnitTe
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
       public void run() {
         CqServiceImpl.EXECUTE_QUERY_DURING_INIT = true;
       }
     });
-    super.tearDown2();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxQueryCQTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxQueryCQTestBase.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxQueryCQTestBase.java
index 71fd4bc..947a4ac 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxQueryCQTestBase.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxQueryCQTestBase.java
@@ -78,7 +78,8 @@ public abstract class PdxQueryCQTestBase extends CacheTestCase {
     return bridgeServerPort;
   }
 
-  public void tearDown2() throws Exception {
+  @Override
+  protected final void preTearDownCacheTestCase() throws Exception {
     disconnectAllFromDS(); // tests all expect to create a new ds
     // Reset the testObject numinstance for the next test.
     TestObject.numInstance = 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryMonitorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryMonitorDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryMonitorDUnitTest.java
index 74183a9..68f0bfa 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryMonitorDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryMonitorDUnitTest.java
@@ -172,14 +172,13 @@ public class QueryMonitorDUnitTest extends CacheTestCase {
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
     Host host = Host.getHost(0);
     disconnectFromDS();
     // shut down clients before servers
     for (int i=numServers; i<4; i++) {
       host.getVM(i).invoke(CacheTestCase.class, "disconnectFromDS");
     }
-    super.tearDown2();
   }
   
   public void createRegion(VM vm){

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/PRDeltaPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/PRDeltaPropagationDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/PRDeltaPropagationDUnitTest.java
index b8399a5..7327e94 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/PRDeltaPropagationDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/PRDeltaPropagationDUnitTest.java
@@ -1142,8 +1142,8 @@ public class PRDeltaPropagationDUnitTest extends DistributedTestCase {
     ConflationDUnitTest.unsetIsSlowStart();
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();    
+  public void preTearDown() throws Exception {
+    super.preTearDown();    
     closeCache();
     client1.invoke(PRDeltaPropagationDUnitTest.class, "closeCache");
     dataStore1.invoke(PRDeltaPropagationDUnitTest.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
index 87fb687..1c9e6d8 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/RemoteCQTransactionDUnitTest.java
@@ -123,13 +123,12 @@ public class RemoteCQTransactionDUnitTest extends CacheTestCase {
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  protected final void preTearDownCacheTestCase() throws Exception {
 //    try { Thread.sleep(5000); } catch (InterruptedException e) { } // FOR MANUAL TESTING OF STATS - DON"T KEEP THIS
     try {
       Invoke.invokeInEveryVM(verifyNoTxState);
     } finally {
       closeAllCache();
-      super.tearDown2();
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
index 9bf1d76..9ce8682 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/CQListGIIDUnitTest.java
@@ -175,8 +175,8 @@ public class CQListGIIDUnitTest extends DistributedTestCase {
   /**
    * Tears down the test.
    */
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     serverVM0.invoke(ConflationDUnitTest.class, "unsetIsSlowStart");
     serverVM1.invoke(ConflationDUnitTest.class, "unsetIsSlowStart");
     closeCache();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADispatcherDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADispatcherDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADispatcherDUnitTest.java
index 74e62b2..4934966 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADispatcherDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADispatcherDUnitTest.java
@@ -160,9 +160,9 @@ public class HADispatcherDUnitTest extends DistributedTestCase
   }
 
   @Override
-  public void tearDown2() throws Exception
+  public void preTearDown() throws Exception
   {
-    super.tearDown2();
+    super.preTearDown();
     client1.invoke(HADispatcherDUnitTest.class, "closeCache");
     client2.invoke(HADispatcherDUnitTest.class, "closeCache");
     // close server

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
index 85a09bd..ac4df17 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientToServerDeltaDUnitTest.java
@@ -133,8 +133,8 @@ public class ClientToServerDeltaDUnitTest extends DistributedTestCase {
     client2 = host.getVM(3);
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     // reset all flags
     DeltaTestImpl.resetDeltaInvokationCounters();
     server.invoke(DeltaTestImpl.class, "resetDeltaInvokationCounters");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
index 72fad35..c467604 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaPropagationWithCQDUnitTest.java
@@ -104,8 +104,8 @@ public class DeltaPropagationWithCQDUnitTest extends DistributedTestCase {
     client2 = host.getVM(3);
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     server1.invoke(DeltaPropagationWithCQDUnitTest.class, "close");
     server2.invoke(DeltaPropagationWithCQDUnitTest.class, "close");
     client1.invoke(DeltaPropagationWithCQDUnitTest.class, "close");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
index 1a8075c..b8a271b 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DeltaToRegionRelationCQRegistrationDUnitTest.java
@@ -590,7 +590,7 @@ public class DeltaToRegionRelationCQRegistrationDUnitTest extends DistributedTes
     return new Integer(p.getPrimaryPort());
   }
   
-  public void tearDown2() throws Exception
+  public void preTearDown() throws Exception
   {
     // donot do any thing as we handling closing cache in test case
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientCrashDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientCrashDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientCrashDUnitTest.java
index 13afbf2..0786be0 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientCrashDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientCrashDUnitTest.java
@@ -43,9 +43,9 @@ public class DurableClientCrashDUnitTest extends DurableClientTestCase {
     this.durableClientVM.invoke(CacheServerTestUtil.class, "setClientCrash", new Object[] {new Boolean(true)});    
   }
   
-  public void tearDown2() throws Exception {
+  public void preTearDown() throws Exception {
     configureClientStop2();
-    super.tearDown2();
+    super.preTearDown();
   }
   
   public void configureClientStop2()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientNetDownDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientNetDownDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientNetDownDUnitTest.java
index 4dd41b8..f58a8a4 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientNetDownDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientNetDownDUnitTest.java
@@ -32,10 +32,10 @@ public class DurableClientNetDownDUnitTest extends DurableClientCrashDUnitTest {
   }
   
   @Override
-  public void tearDown2() throws Exception {
+  public void preTearDown() throws Exception {
     //ensure that the test flag is no longer set in this vm
     this.durableClientVM.invoke(CacheServerTestUtil.class, "reconnectClient");
-    super.tearDown2();
+    super.preTearDown();
   }
 
   public void setPrimaryRecoveryCheck() {}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
index ec202e3..26650bf 100755
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientTestCase.java
@@ -101,11 +101,12 @@ public class DurableClientTestCase extends DistributedTestCase {
     this.regionName = getName() + "_region";
     //Clients see this when the servers disconnect
     IgnoredException.addIgnoredException("Could not find any server");
-    testName = getName();
-    System.out.println("\n\n[setup] START TEST " + getClass().getSimpleName()+"."+ testName+"\n\n");
+    setTestMethodName(getName());
+    assertEquals(getName(), getTestMethodName());
+    System.out.println("\n\n[setup] START TEST " + getClass().getSimpleName()+"."+ getTestMethodName()+"\n\n");
   }
   
-  public void tearDown2() throws Exception {
+  public void preTearDown() throws Exception {
     this.durableClientVM.invoke(CacheServerTestUtil.class, "closeCache");
     this.publisherClientVM.invoke(CacheServerTestUtil.class, "closeCache");
     this.server1VM.invoke(CacheServerTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
index 81f429f..a5b3dd8 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
@@ -93,8 +93,8 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     super.setUp();
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     disconnectAllFromDS();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
index 8471b3b..b4cb4f1 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
@@ -1436,8 +1436,8 @@ private void setUpNonSubscribedClient() throws Exception {
     });
   }
   
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  @Override
+  protected final void postTearDownCacheTestCase() throws Exception {
     Host.getHost(0).getVM(0).invoke(CacheServerTestUtil.class, "closeCache");
     Host.getHost(0).getVM(1).invoke(CacheServerTestUtil.class, "closeCache");
     Host.getHost(0).getVM(2).invoke(CacheServerTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
index 104abc7..3bce205 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
@@ -427,8 +427,8 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
     return p;
   }
   
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  @Override
+  protected final void postTearDownCacheTestCase() throws Exception {
     Host.getHost(0).getVM(0).invoke(CacheServerTestUtil.class, "closeCache");
     Host.getHost(0).getVM(1).invoke(CacheServerTestUtil.class, "closeCache");
     Host.getHost(0).getVM(2).invoke(CacheServerTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestCQDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestCQDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestCQDUnitTest.java
index 833d49b..75a41b6 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestCQDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestCQDUnitTest.java
@@ -51,8 +51,8 @@ public class TestCQDUnitTest extends ManagementTestBase {
 
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
   }
 
   public static long getNumOfCQ() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestClientsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestClientsDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestClientsDUnitTest.java
index 552ecb9..0e23900 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestClientsDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestClientsDUnitTest.java
@@ -48,8 +48,8 @@ public class TestClientsDUnitTest extends ManagementTestBase {
 
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
   }
 
   public static Integer getNumOfClients() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestServerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestServerDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestServerDUnitTest.java
index 721e78a..fce6d91 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestServerDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/management/internal/pulse/TestServerDUnitTest.java
@@ -45,8 +45,8 @@ public class TestServerDUnitTest extends ManagementTestBase {
 
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
   }
 
   public static int getNumOfServersFromMBean() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
index 6ccc27e..b464451 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
@@ -231,9 +231,9 @@ public class ClientAuthorizationTwoDUnitTest extends
   // End Region: Tests
 
   @Override
-  public void tearDown2() throws Exception {
+  public void preTearDown() throws Exception {
 
-    super.tearDown2();
+    super.preTearDown();
     // close the clients first
     client1.invoke(SecurityTestUtil.class, "closeCache");
     client2.invoke(SecurityTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
index bf57be6..b34b4a2 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
@@ -140,8 +140,8 @@ public class ClientAuthzObjectModDUnitTest extends ClientAuthorizationTestBase {
     server2.invoke(registerInstantiator);
   }
   
-  public void tearDown2() throws Exception  {
-    super.tearDown2();
+  public void preTearDown() throws Exception  {
+    super.preTearDown();
     DistributedTestCase.cleanupAllVms();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
index 0e4dcce..510b134 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
@@ -95,8 +95,8 @@ public class ClientCQPostAuthorizationDUnitTest extends
     SecurityTestUtil.registerExpectedExceptions(clientExpectedExceptions);
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     client1.invoke(SecurityTestUtil.class, "closeCache");
     client2.invoke(SecurityTestUtil.class, "closeCache");
     server1.invoke(SecurityTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
index 6642124..b304885 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
@@ -384,9 +384,9 @@ public class ClientPostAuthorizationDUnitTest extends
 
   // End Region: Tests
 
-  public void tearDown2() throws Exception {
+  public void preTearDown() throws Exception {
 
-    super.tearDown2();
+    super.preTearDown();
     // close the clients first
     client1.invoke(SecurityTestUtil.class, "closeCache");
     client2.invoke(SecurityTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-cq/src/test/java/com/gemstone/gemfire/security/MultiuserAPIDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/MultiuserAPIDUnitTest.java b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/MultiuserAPIDUnitTest.java
index 5997562..b13da33 100644
--- a/gemfire-cq/src/test/java/com/gemstone/gemfire/security/MultiuserAPIDUnitTest.java
+++ b/gemfire-cq/src/test/java/com/gemstone/gemfire/security/MultiuserAPIDUnitTest.java
@@ -380,8 +380,8 @@ public class MultiuserAPIDUnitTest extends ClientAuthorizationTestBase {
     }
   }
 
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     // close the clients first
     client1.invoke(SecurityTestUtil.class, "closeCache");
     client2.invoke(SecurityTestUtil.class, "closeCache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c0552f26/gemfire-wan/src/test/java/com/gemstone/gemfire/internal/cache/UpdateVersionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-wan/src/test/java/com/gemstone/gemfire/internal/cache/UpdateVersionDUnitTest.java b/gemfire-wan/src/test/java/com/gemstone/gemfire/internal/cache/UpdateVersionDUnitTest.java
index f4e87f8..5b2dda1 100644
--- a/gemfire-wan/src/test/java/com/gemstone/gemfire/internal/cache/UpdateVersionDUnitTest.java
+++ b/gemfire-wan/src/test/java/com/gemstone/gemfire/internal/cache/UpdateVersionDUnitTest.java
@@ -80,8 +80,8 @@ public class UpdateVersionDUnitTest extends DistributedTestCase {
     super(name);
   }
   
-  public void tearDown2() throws Exception {
-    super.tearDown2();
+  public void preTearDown() throws Exception {
+    super.preTearDown();
     closeCache();
     Invoke.invokeInEveryVM(new SerializableRunnable() { public void run() {
       closeCache();
@@ -651,7 +651,7 @@ public class UpdateVersionDUnitTest extends DistributedTestCase {
    */
 
   private static void createCache(Integer locPort) {
-    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(testName);
+    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(getTestMethodName());
     Properties props = new Properties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort + "]");
@@ -790,7 +790,7 @@ public class UpdateVersionDUnitTest extends DistributedTestCase {
   }
 
   public static Integer createFirstRemoteLocator(int dsId, int remoteLocPort) {
-    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(testName);
+    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(getTestMethodName());
     int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
     Properties props = new Properties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME,"0");
@@ -858,7 +858,7 @@ public class UpdateVersionDUnitTest extends DistributedTestCase {
   }
 
   public static int createReceiver(int locPort) {
-    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(testName);
+    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(getTestMethodName());
     Properties props = new Properties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
@@ -950,7 +950,7 @@ public class UpdateVersionDUnitTest extends DistributedTestCase {
   }
 
   public static Integer createFirstLocatorWithDSId(int dsId) {
-    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(testName);
+    UpdateVersionDUnitTest test = new UpdateVersionDUnitTest(getTestMethodName());
     int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
     Properties props = new Properties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME,"0");