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/02 21:21:30 UTC

[23/44] incubator-geode git commit: Refactor dunit

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
index 7dc8e4b..d732a56 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
@@ -50,6 +50,8 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * @author dsmith
@@ -109,7 +111,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
         af.setPartitionAttributes(paf.create());
         
         //Try to colocate a persistent PR with the non persistent PR. This should fail.
-        IgnoredException exp = IgnoredException.addExpectedException("IllegalStateException");
+        IgnoredException exp = IgnoredException.addIgnoredException("IllegalStateException");
         try {
           cache.createRegion("colocated", af.create());
           fail("should not have been able to create a persistent region colocated with a non persistent region");
@@ -292,7 +294,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     vm1.invoke(createChildPR);
     vm2.invoke(createChildPR);
     
-    pause(4000);
+    Wait.pause(4000);
     
     assertEquals(vm0Buckets, getBucketList(vm0, PR_REGION_NAME));
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
@@ -604,7 +606,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     
     createData(vm0, 0, NUM_BUCKETS, "b");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
-    IgnoredException expected = IgnoredException.addExpectedException("PartitionOfflineException");
+    IgnoredException expected = IgnoredException.addIgnoredException("PartitionOfflineException");
     try {
     
     //Close the remaining members.
@@ -632,7 +634,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     //by starting it last.
     AsyncInvocation async0 = vm0.invokeAsync(createPRs);
     AsyncInvocation async1 = vm1.invokeAsync(createPRs);
-    pause(2000);
+    Wait.pause(2000);
     AsyncInvocation async2 = vm2.invokeAsync(createPRs);
     async0.getResult(MAX_WAIT);
     async1.getResult(MAX_WAIT);
@@ -806,8 +808,8 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
    * @throws Throwable 
    */
   public void replaceOfflineMemberAndRestartCreateColocatedPRLate(SerializableRunnable createParentPR, SerializableRunnable createChildPR) throws Throwable {
-    IgnoredException.addExpectedException("PartitionOfflineException");
-    IgnoredException.addExpectedException("RegionDestroyedException");
+    IgnoredException.addIgnoredException("PartitionOfflineException");
+    IgnoredException.addIgnoredException("RegionDestroyedException");
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
@@ -862,14 +864,14 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     //by starting it last.
     AsyncInvocation async2 = vm2.invokeAsync(createParentPR);
     AsyncInvocation async1 = vm1.invokeAsync(createParentPR);
-    pause(2000);
+    Wait.pause(2000);
     AsyncInvocation async0 = vm0.invokeAsync(createParentPR);
     async0.getResult(MAX_WAIT);
     async1.getResult(MAX_WAIT);
     async2.getResult(MAX_WAIT);
     
     //Wait for async tasks
-    pause(2000);
+    Wait.pause(2000);
     
     //Recreate the child region. 
     async2 = vm2.invokeAsync(createChildPR);
@@ -977,7 +979,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
       }
     });
     
-    IgnoredException ex = IgnoredException.addExpectedException("PartitionOfflineException", vm1);
+    IgnoredException ex = IgnoredException.addIgnoredException("PartitionOfflineException", vm1);
     try {
       
     //Do a rebalance to create buckets in vm1. THis will cause vm0 to disconnect
@@ -997,7 +999,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     vm0.invoke(new SerializableCallable() {
       
       public Object call() throws Exception {
-        waitForCriterion(new WaitCriterion() {
+        Wait.waitForCriterion(new WaitCriterion() {
           public boolean done() {
             InternalDistributedSystem ds = system;
             return ds == null || !ds.isConnected();
@@ -1417,7 +1419,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     closeCache();
     
     //Restart colocated with "region2"
-    IgnoredException ex = IgnoredException.addExpectedException("DiskAccessException|IllegalStateException");
+    IgnoredException ex = IgnoredException.addIgnoredException("DiskAccessException|IllegalStateException");
     try {
       createColocatedPRs("region2");
       fail("Should have received an illegal state exception");
@@ -1439,7 +1441,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest extends
     
     //Restart uncolocated. We don't allow changing
     //from uncolocated to colocated.
-    ex = IgnoredException.addExpectedException("DiskAccessException|IllegalStateException");
+    ex = IgnoredException.addIgnoredException("DiskAccessException|IllegalStateException");
     try {
       createColocatedPRs(null);
       fail("Should have received an illegal state exception");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
index 2ef4441..4364b0f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
@@ -78,11 +78,15 @@ import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.LogWriterSupport;
+import com.gemstone.gemfire.test.dunit.NetworkSupport;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.RMIException;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * Tests the basic use cases for PR persistence.
@@ -145,8 +149,8 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     createPR(vm0, 0, 0, 5);
     createData(vm0, 0, 5, "a");
     closeCache(vm0);
-    IgnoredException expect = IgnoredException.addExpectedException("IllegalStateException", vm0);
-    expect = IgnoredException.addExpectedException("DiskAccessException", vm0);
+    IgnoredException expect = IgnoredException.addIgnoredException("IllegalStateException", vm0);
+    expect = IgnoredException.addIgnoredException("DiskAccessException", vm0);
     try {
       createPR(vm0, 0, 0, 2);
       fail("Expect to get java.lang.IllegalStateException, but it did not");
@@ -256,8 +260,8 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       }
     });
 
-    IgnoredException expected1 = IgnoredException.addExpectedException("Fatal error from asynch");
-    IgnoredException expected2 = IgnoredException.addExpectedException("ToDataException");
+    IgnoredException expected1 = IgnoredException.addIgnoredException("Fatal error from asynch");
+    IgnoredException expected2 = IgnoredException.addIgnoredException("ToDataException");
     try {
       int redundancy=1;
       createPR(vm0, redundancy, -1, 113, false);
@@ -424,7 +428,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     createData(vm0, numBuckets, 113, "b");
     checkData(vm0, numBuckets, 113, "b");
     
-    IgnoredException ex = IgnoredException.addExpectedException(RevokedPersistentDataException.class.getName(), vm1);
+    IgnoredException ex = IgnoredException.addIgnoredException(RevokedPersistentDataException.class.getName(), vm1);
     try {
       createPR(vm1, 1);
       fail("Should have recieved a SplitDistributedSystemException");
@@ -439,7 +443,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
   }
   
   public void testRevokeBeforeStartup() throws Throwable {
-    IgnoredException.addExpectedException("RevokeFailedException");
+    IgnoredException.addIgnoredException("RevokeFailedException");
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
@@ -493,7 +497,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     createData(vm0, numBuckets, 113, "b");
     checkData(vm0, numBuckets, 113, "b");
     
-    IgnoredException ex = IgnoredException.addExpectedException(RevokedPersistentDataException.class.getName(), vm1);
+    IgnoredException ex = IgnoredException.addIgnoredException(RevokedPersistentDataException.class.getName(), vm1);
     try {
       createPR(vm1, 1);
       fail("Should have recieved a SplitDistributedSystemException");
@@ -539,7 +543,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     final int aVM1Bucket = vm1Buckets.iterator().next();
     closeCache(vm1);
 
-    IgnoredException ex = IgnoredException.addExpectedException("PartitionOfflineException");
+    IgnoredException ex = IgnoredException.addIgnoredException("PartitionOfflineException");
     try { 
       checkReadWriteOperationsWithOfflineMember(vm0, aVM0Bucket, aVM1Bucket);
       //Make sure that a newly created member is informed about the offline member
@@ -576,7 +580,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       }
     }
 
-    IgnoredException expect = IgnoredException.addExpectedException("PartitionOfflineException", vm0);
+    IgnoredException expect = IgnoredException.addIgnoredException("PartitionOfflineException", vm0);
     //Try a function execution
     vm0.invoke(new SerializableRunnable("Test ways to read") {
       public void run() {
@@ -805,7 +809,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       //This should work, because this bucket is still available.
       checkData(vm0, aVM0Bucket, aVM0Bucket + 1, "a");
       
-      IgnoredException expect = IgnoredException.addExpectedException("PartitionOfflineException", vm0);
+      IgnoredException expect = IgnoredException.addIgnoredException("PartitionOfflineException", vm0);
       try {
         checkData(vm0, aVM1Bucket, aVM1Bucket + 1, "a");
         fail("Should not have been able to read from missing buckets!");
@@ -838,7 +842,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       createData(vm2, aVM1Bucket, aVM1Bucket + 1, "a");
       checkData(vm2, aVM1Bucket, aVM1Bucket + 1, "a");
       
-      IgnoredException ex = IgnoredException.addExpectedException(RevokedPersistentDataException.class.getName(), vm1);
+      IgnoredException ex = IgnoredException.addIgnoredException(RevokedPersistentDataException.class.getName(), vm1);
       try {
         createPR(vm1, 0);
         fail("Should have recieved a RevokedPersistentDataException");
@@ -886,7 +890,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       Set<Integer> vm2Buckets = getBucketList(vm2);
       assertEquals(vm1Buckets, vm2Buckets);
       
-      IgnoredException ex = IgnoredException.addExpectedException(RevokedPersistentDataException.class.getName(), vm1);
+      IgnoredException ex = IgnoredException.addIgnoredException(RevokedPersistentDataException.class.getName(), vm1);
       try {
         createPR(vm1, 1);
         fail("Should have recieved a SplitDistributedSystemException");
@@ -954,7 +958,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       
       //VM2 should pick up the slack
       
-      waitForCriterion(new WaitCriterion() {
+      Wait.waitForCriterion(new WaitCriterion() {
         
         public boolean done() {
           Set<Integer> vm2Buckets = getBucketList(vm2);
@@ -1338,11 +1342,11 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
   
   public void testRegisterInterestNoDataStores() {
     //Closing the client may log a warning on the server
-    IgnoredException.addExpectedException("Connection reset");
-    IgnoredException.addExpectedException("SocketTimeoutException");
-    IgnoredException.addExpectedException("ServerConnectivityException");
-    IgnoredException.addExpectedException("Socket Closed");
-    IgnoredException.addExpectedException("Unexpected IOException");
+    IgnoredException.addIgnoredException("Connection reset");
+    IgnoredException.addIgnoredException("SocketTimeoutException");
+    IgnoredException.addIgnoredException("ServerConnectivityException");
+    IgnoredException.addIgnoredException("Socket Closed");
+    IgnoredException.addIgnoredException("Unexpected IOException");
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
@@ -1383,7 +1387,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
           Cache cache = getCache();
 
           PoolFactory pf = PoolManager.createFactory();
-          pf.addServer(getServerHostName(host), serverPort);
+          pf.addServer(NetworkSupport.getServerHostName(host), serverPort);
           pf.setSubscriptionEnabled(true);
           pf.create("pool");
           AttributesFactory af = new AttributesFactory();
@@ -1446,7 +1450,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
               DistributedTestCase.disconnectFromDS();
               
               await().atMost(30, SECONDS).until(() -> {return (cache == null || cache.isClosed());});
-              getLogWriter().info("Cache is confirmed closed");
+              LogWriterSupport.getLogWriter().info("Cache is confirmed closed");
             }
           }
         });
@@ -1657,7 +1661,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       }
     });
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPR(vm0, 1, 0, 1);
     
     //Make sure we create a bucket
@@ -1665,9 +1669,9 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     
     //This should recover redundancy, which should cause vm0 to disconnect
     
-    IgnoredException ex = IgnoredException.addExpectedException("PartitionOfflineException");
+    IgnoredException ex = IgnoredException.addIgnoredException("PartitionOfflineException");
     try { 
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPR(vm1, 1, 0, 1);
     
     //Make sure get a partition offline exception
@@ -1823,7 +1827,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
    * to make sure that later we can recover redundancy.
    */
   public void testCrashDuringBucketGII() {
-    IgnoredException.addExpectedException("PartitionOfflineException");
+    IgnoredException.addIgnoredException("PartitionOfflineException");
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
@@ -1877,7 +1881,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
    * @throws InterruptedException 
    */
   public void testCrashDuringBucketGII2() throws InterruptedException {
-    IgnoredException.addExpectedException("PartitionOfflineException");
+    IgnoredException.addIgnoredException("PartitionOfflineException");
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     final VM vm1 = host.getVM(1);
@@ -1935,7 +1939,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
         return null;
       }
     };
-    DistributedTestCase.waitForCriterion(ev, 30 * 1000, 200, true);
+    Wait.waitForCriterion(ev, 30 * 1000, 200, true);
     assertEquals(Collections.singleton(0), getBucketList(vm1));
   }
   
@@ -1958,7 +1962,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     createData(vm1, 1, 2, "a");
     
     //this should throw a conflicting data exception.
-    IgnoredException expect = IgnoredException.addExpectedException("ConflictingPersistentDataException", vm0);
+    IgnoredException expect = IgnoredException.addIgnoredException("ConflictingPersistentDataException", vm0);
     try {
       createPR(vm0, 0);
       fail("should have seen a conflicting data exception");
@@ -1976,7 +1980,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     //view from vm0 because vm0 was in conflict!
     //In fact, this is a bit of a problem, because now vm1 is dependent
     //on vm vm0.
-    expect = IgnoredException.addExpectedException("PartitionOfflineException", vm1);
+    expect = IgnoredException.addIgnoredException("PartitionOfflineException", vm1);
     try {
       createData(vm1, 0, 1, "a");
       fail("Should have seen a PartitionOfflineException for bucket 0");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
index eae178f..17cd552 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
@@ -58,9 +58,13 @@ import com.gemstone.gemfire.internal.cache.persistence.PersistenceAdvisor;
 import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.LogWriterSupport;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * @author dsmith
@@ -85,7 +89,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
   public void setUp() throws Exception {
     super.setUp();
     disconnectAllFromDS();
-    invokeInEveryVM(PersistentPartitionedRegionTestBase.class,"setRegionName", new Object[]{getUniqueName()});
+    Invoke.invokeInEveryVM(PersistentPartitionedRegionTestBase.class,"setRegionName", new Object[]{getUniqueName()});
     setRegionName(getUniqueName());
   }
   
@@ -199,7 +203,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
 
   protected void createData(VM vm, final int startKey, final int endKey,
       final String value) {
-    getLogWriter().info("createData invoked.  PR_REGION_NAME is " + PR_REGION_NAME);
+    LogWriterSupport.getLogWriter().info("createData invoked.  PR_REGION_NAME is " + PR_REGION_NAME);
         createData(vm, startKey, endKey,value, PR_REGION_NAME);
       }
 
@@ -209,7 +213,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
           
           public void run() {
             Cache cache = getCache();
-            getLogWriter().info("creating data in " + regionName);
+            LogWriterSupport.getLogWriter().info("creating data in " + regionName);
             Region region = cache.getRegion(regionName);
             
             for(int i =startKey; i < endKey; i++) {
@@ -475,7 +479,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
       public Object call() throws Exception {
         Cache cache = getCache();
         final PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
-        waitForCriterion(new WaitCriterion() {
+        Wait.waitForCriterion(new WaitCriterion() {
           
           public boolean done() {
             return expectedBuckets.equals(getActualBuckets());
@@ -554,7 +558,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
                 }
               }
             };
-            waitForCriterion(wc, MAX_WAIT, 500, true);
+            Wait.waitForCriterion(wc, MAX_WAIT, 500, true);
           } finally {
             adminDS.disconnect();
           }
@@ -686,7 +690,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
         Cache cache = getCache();
         PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
         final PartitionedRegionDataStore dataStore = region.getDataStore();
-        waitForCriterion(new WaitCriterion() {
+        Wait.waitForCriterion(new WaitCriterion() {
     
           public boolean done() {
             Set<Integer> vm2Buckets = dataStore.getAllLocalBucketIds();
@@ -708,7 +712,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
       public void run() {
         Cache cache = getCache();
         final Region region = cache.getRegion(regionName);
-        waitForCriterion(new WaitCriterion() {
+        Wait.waitForCriterion(new WaitCriterion() {
           
           public boolean done() {
             PartitionRegionInfo info = PartitionRegionHelper.getPartitionRegionInfo(region);
@@ -786,7 +790,7 @@ public abstract class PersistentPartitionedRegionTestBase extends CacheTestCase
     BufferedReader br = new BufferedReader(new InputStreamReader(is));
     String line;
     while((line = br.readLine()) != null) {
-      getLogWriter().fine("OUTPUT:" + line);
+      LogWriterSupport.getLogWriter().fine("OUTPUT:" + line);
       //TODO validate output
     };
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
index e92d6b1..eab623a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
@@ -23,6 +23,8 @@ import com.gemstone.gemfire.internal.cache.DiskStoreImpl;
 import com.gemstone.gemfire.internal.cache.TXManagerImpl;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.LogWriterSupport;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 
@@ -42,7 +44,7 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest extends Persist
   @Override
   public void tearDown2() throws Exception {
     super.tearDown2();
-    invokeInEveryVM(new SerializableRunnable() {
+    Invoke.invokeInEveryVM(new SerializableRunnable() {
       
       public void run() {
         TXManagerImpl.ALLOW_PERSISTENT_TRANSACTIONS = false;
@@ -57,7 +59,7 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest extends Persist
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    invokeInEveryVM(new SerializableRunnable() {
+    Invoke.invokeInEveryVM(new SerializableRunnable() {
       
       public void run() {
         TXManagerImpl.ALLOW_PERSISTENT_TRANSACTIONS = true;
@@ -134,12 +136,12 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest extends Persist
   @Override
   protected void createData(VM vm, final int startKey, final int endKey, final String value,
       final String regionName) {
-    getLogWriter().info("creating runnable to create data for region " + regionName);
+    LogWriterSupport.getLogWriter().info("creating runnable to create data for region " + regionName);
     SerializableRunnable createData = new SerializableRunnable() {
       
       public void run() {
         Cache cache = getCache();
-        getLogWriter().info("getting region " + regionName);
+        LogWriterSupport.getLogWriter().info("getting region " + regionName);
         Region region = cache.getRegion(regionName);
         
         for(int i =startKey; i < endKey; i++) {
@@ -171,7 +173,7 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest extends Persist
       
       public void run() {
         Cache cache = getCache();
-        getLogWriter().info("checking data in " + regionName);
+        LogWriterSupport.getLogWriter().info("checking data in " + regionName);
         Region region = cache.getRegion(regionName);
         
         for(int i =startKey; i < endKey; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
index 0c22fd8..cd68c35 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
@@ -59,10 +59,12 @@ import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.RMIException;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
 
 /**
  * Tests the basic use cases for PR persistence.
@@ -143,7 +145,7 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     createData(vm0, 0, numBuckets, "a", "region");
 
     vm0.invoke(addExceptionTag1(expectedExceptions));
-    invokeInEveryVM(new SerializableRunnable("set TestInternalGemFireError") {
+    Invoke.invokeInEveryVM(new SerializableRunnable("set TestInternalGemFireError") {
       public void run() {
         System.setProperty("TestInternalGemFireError", "true");
       }
@@ -152,7 +154,7 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     
     assertTrue(InternalDistributedSystem.getExistingSystems().isEmpty());
     
-    invokeInEveryVM(new SerializableRunnable("reset TestInternalGemFireError") {
+    Invoke.invokeInEveryVM(new SerializableRunnable("reset TestInternalGemFireError") {
       public void run() {
         System.setProperty("TestInternalGemFireError", "false");
       }
@@ -179,7 +181,7 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     
     vm0.invoke(addExceptionTag1(expectedExceptions));
     vm1.invoke(addExceptionTag1(expectedExceptions));
-    invokeInEveryVM(new SerializableRunnable("set TestInternalGemFireError") {
+    Invoke.invokeInEveryVM(new SerializableRunnable("set TestInternalGemFireError") {
       public void run() {
         System.setProperty("TestInternalGemFireError", "true");
       }
@@ -188,7 +190,7 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     
     assertTrue(InternalDistributedSystem.getExistingSystems().isEmpty());
     
-    invokeInEveryVM(new SerializableRunnable("reset TestInternalGemFireError") {
+    Invoke.invokeInEveryVM(new SerializableRunnable("reset TestInternalGemFireError") {
       public void run() {
         System.setProperty("TestInternalGemFireError", "false");
       }
@@ -484,7 +486,7 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     });
     
     //wait for shutdown to finish
-    pause(10000);
+    Wait.pause(10000);
     
     // restart vm0
     AsyncInvocation a0 = createRegionAsync(vm0, "region", "disk", true, 1);
@@ -547,7 +549,7 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     AsyncInvocation a0 = createRegionAsync(vm0, "region", "disk", true, 1);
     
     //Wait a bit for the initialization to get stuck
-    pause(20000);
+    Wait.pause(20000);
     assertTrue(a0.isAlive());
     
     //Do another shutdown all, with a member offline and another stuck

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningDUnitTest.java
index 2d2e886..f6d6c13 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningDUnitTest.java
@@ -26,6 +26,7 @@ import com.gemstone.gemfire.cache.partition.PartitionNotAvailableException;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.Host;
 
 /**
@@ -148,7 +149,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
    */
 
   public void testSamePartitionName_Primary_OnTwoMembers() {
-    IgnoredException ex = IgnoredException.addExpectedException("DuplicatePrimaryPartitionException");
+    IgnoredException ex = IgnoredException.addIgnoredException("DuplicatePrimaryPartitionException");
     try {
       member1.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
       FixedPartitionAttributes fpa1 = FixedPartitionAttributes
@@ -193,7 +194,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
    */
 
   public void testSamePartitionName_DifferentNumBuckets() {
-    IgnoredException ex = IgnoredException.addExpectedException("IllegalStateException");
+    IgnoredException ex = IgnoredException.addIgnoredException("IllegalStateException");
     try {
       member1.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
       FixedPartitionAttributes fpa1 = FixedPartitionAttributes
@@ -236,7 +237,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
    */
 
   public void testNumberOfPartitions() {
-    IgnoredException expected = IgnoredException.addExpectedException("IllegalStateException");
+    IgnoredException expected = IgnoredException.addIgnoredException("IllegalStateException");
     try {
       member1.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
       member1.invoke(FixedPartitioningTestBase.class, "createRegionWithPartitionAttributes",
@@ -296,7 +297,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
    */
 
   public void testNumBuckets_totalNumBuckets() {
-    IgnoredException expected = IgnoredException.addExpectedException("IllegalStateException");
+    IgnoredException expected = IgnoredException.addIgnoredException("IllegalStateException");
     try {
       member1.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
       member1.invoke(FixedPartitioningTestBase.class, "createRegionWithPartitionAttributes",
@@ -385,7 +386,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
    */
   
   public void test_DataStoreWithoutPartition_DataStoreWithPartition() {
-    IgnoredException expected = IgnoredException.addExpectedException("IllegalStateException");
+    IgnoredException expected = IgnoredException.addIgnoredException("IllegalStateException");
     try {
       member1.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
       member1.invoke(FixedPartitioningTestBase.class, "createRegionWithPartitionAttributes",
@@ -417,7 +418,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
    */
 
   public void test_DataStoreWithPartition_DataStoreWithoutPartition() {
-    IgnoredException expected = IgnoredException.addExpectedException("IllegalStateException");
+    IgnoredException expected = IgnoredException.addIgnoredException("IllegalStateException");
     try {
       member2.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
       FixedPartitionAttributes fpa1 = FixedPartitionAttributes
@@ -1209,7 +1210,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
     createRegionWithPartitionAttributes("Quarter", fpaList, 3, 40, 12,
         new QuarterPartitionResolver(), null, false);
 
-    pause(1000);
+    Wait.pause(1000);
 
     member1.invoke(FixedPartitioningTestBase.class, "checkPrimaryBucketsForQuarter",
         new Object[] { 3, 0 });
@@ -1329,7 +1330,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
         new Object[] { 9, 3 });
 
     member4.invoke(FixedPartitioningTestBase.class, "closeCache");
-    pause(1000);
+    Wait.pause(1000);
 
     member1.invoke(FixedPartitioningTestBase.class,
         "checkPrimarySecondaryData_TwoSecondaries", new Object[] { Quarter1,
@@ -1439,7 +1440,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
         new Object[] { 6, 3 });
 
     member4.invoke(FixedPartitioningTestBase.class, "closeCache");
-    pause(1000);
+    Wait.pause(1000);
 
     member3.invoke(FixedPartitioningTestBase.class, "checkPrimaryBucketsForQuarterAfterCacheClosed",
         new Object[] { 6, 6 });
@@ -1457,7 +1458,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
         new Object[] { "Quarter", fpaList, 1, 40, 12,
             new QuarterPartitionResolver(), null, false });
 
-    pause(1000);
+    Wait.pause(1000);
 
     member1.invoke(FixedPartitioningTestBase.class, "checkPrimarySecondaryData",
         new Object[] { Quarter1, true });
@@ -1544,7 +1545,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
 
     member4.invoke(FixedPartitioningTestBase.class, "closeCache");
     member2.invoke(FixedPartitioningTestBase.class, "closeCache");
-    pause(1000);
+    Wait.pause(1000);
 
     member3.invoke(FixedPartitioningTestBase.class, "checkPrimaryBucketsForQuarterAfterCacheClosed",
         new Object[] { 6, 6 });
@@ -1573,7 +1574,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
     member2.invoke(FixedPartitioningTestBase.class, "createRegionWithPartitionAttributes",
         new Object[] { "Quarter", fpaList, 1, 40, 12,
             new QuarterPartitionResolver(), null, false });
-    pause(1000);
+    Wait.pause(1000);
 
     member1.invoke(FixedPartitioningTestBase.class, "checkPrimarySecondaryData",
         new Object[] { Quarter1, true });
@@ -1595,7 +1596,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
     
     member4.invoke(FixedPartitioningTestBase.class, "doRebalance");
     
-    pause(2000);
+    Wait.pause(2000);
     member1.invoke(FixedPartitioningTestBase.class, "checkPrimaryBucketsForQuarter",
         new Object[] { 6, 3 });
     member2.invoke(FixedPartitioningTestBase.class, "checkPrimaryBucketsForQuarter",
@@ -1675,7 +1676,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
 
     member3.invoke(FixedPartitioningTestBase.class, "closeCache");
     
-    pause(1000);  
+    Wait.pause(1000);  
     
     member1.invoke(FixedPartitioningTestBase.class, "checkStartingBucketIDs_Nodedown");
     member2.invoke(FixedPartitioningTestBase.class, "checkStartingBucketIDs_Nodedown");
@@ -1686,7 +1687,7 @@ public class FixedPartitioningDUnitTest extends FixedPartitioningTestBase {
         new Object[] { "Quarter", fpaList, 2, 40, 12,
             new QuarterPartitionResolver(), null, false });
     
-    pause(3000);
+    Wait.pause(3000);
     
     member1.invoke(FixedPartitioningTestBase.class, "checkStartingBucketIDs_Nodeup");
     member2.invoke(FixedPartitioningTestBase.class, "checkStartingBucketIDs_Nodeup");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
index 0fba3c8..c18e2a9 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
@@ -73,8 +73,10 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.LogWriterSupport;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * This is the base class to do operations
@@ -461,7 +463,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
             "putCustomerPartitionedRegion : failed while doing put operation in CustomerPartitionedRegion ",
             e);
       }
-      getLogWriter().info("Customer :- { " + custid + " : " + customer + " }");
+      LogWriterSupport.getLogWriter().info("Customer :- { " + custid + " : " + customer + " }");
     }
   }
   
@@ -485,7 +487,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
               "putOrderPartitionedRegion : failed while doing put operation in OrderPartitionedRegion ",
               e);
         }
-        getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
+        LogWriterSupport.getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
       }
     }
   }
@@ -512,7 +514,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                 "putShipmentPartitionedRegion : failed while doing put operation in ShipmentPartitionedRegion ",
                 e);
           }
-          getLogWriter().info(
+          LogWriterSupport.getLogWriter().info(
               "Shipment :- { " + shipmentId + " : " + shipment + " }");
         }
       }
@@ -536,7 +538,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
             "putCustomerPartitionedRegion : failed while doing put operation in CustomerPartitionedRegion ",
             e);
       }
-      getLogWriter().info("Customer :- { " + custid + " : " + customer + " }");
+      LogWriterSupport.getLogWriter().info("Customer :- { " + custid + " : " + customer + " }");
     }
   }
   
@@ -560,7 +562,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
               "putOrderPartitionedRegion : failed while doing put operation in OrderPartitionedRegion ",
               e);
         }
-        getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
+        LogWriterSupport.getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
       }
     }
   }
@@ -587,7 +589,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                 "putShipmentPartitionedRegion : failed while doing put operation in ShipmentPartitionedRegion ",
                 e);
           }
-          getLogWriter().info(
+          LogWriterSupport.getLogWriter().info(
               "Shipment :- { " + shipmentId + " : " + shipment + " }");
         }
       }
@@ -612,7 +614,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
               "putCustomerPartitionedRegion : failed while doing put operation in CustomerPartitionedRegion ",
               e);
         }
-        getLogWriter()
+        LogWriterSupport.getLogWriter()
             .info("Customer :- { " + custid + " : " + customer + " }");
       }
     }
@@ -639,7 +641,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                 "putOrderPartitionedRegion : failed while doing put operation in OrderPartitionedRegion ",
                 e);
           }
-          getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
+          LogWriterSupport.getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
         }
       }
     }
@@ -668,7 +670,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                   "putShipmentPartitionedRegion : failed while doing put operation in ShipmentPartitionedRegion ",
                   e);
             }
-            getLogWriter().info(
+            LogWriterSupport.getLogWriter().info(
                 "Shipment :- { " + shipmentId + " : " + shipment + " }");
           }
         }
@@ -694,7 +696,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
               "putCustomerPartitionedRegion : failed while doing put operation in CustomerPartitionedRegion ",
               e);
         }
-        getLogWriter()
+        LogWriterSupport.getLogWriter()
             .info("Customer :- { " + custid + " : " + customer + " }");
       }
     }
@@ -721,7 +723,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                 "putOrderPartitionedRegion : failed while doing put operation in OrderPartitionedRegion ",
                 e);
           }
-          getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
+          LogWriterSupport.getLogWriter().info("Order :- { " + orderId + " : " + order + " }");
         }
       }
     }
@@ -750,7 +752,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                   "putShipmentPartitionedRegion : failed while doing put operation in ShipmentPartitionedRegion ",
                   e);
             }
-            getLogWriter().info(
+            LogWriterSupport.getLogWriter().info(
                 "Shipment :- { " + shipmentId + " : " + shipment + " }");
           }
         }
@@ -851,7 +853,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
           // assertNotNull(orderPartitionedregion.get(orderId));
 
           if (custId.equals(orderId.getCustId())) {
-            getLogWriter().info(
+            LogWriterSupport.getLogWriter().info(
                 orderId + "belongs to node " + idmForCustomer + " "
                     + idmForOrder);
             assertEquals(idmForCustomer, idmForOrder);
@@ -863,7 +865,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
             ShipmentId shipmentId = (ShipmentId)shipmentIterator.next();
             // assertNotNull(shipmentPartitionedregion.get(shipmentId));
             if (orderId.equals(shipmentId.getOrderId())) {
-              getLogWriter().info(
+              LogWriterSupport.getLogWriter().info(
                   shipmentId + "belongs to node " + idmForOrder + " "
                       + idmForShipment);
             }
@@ -1087,15 +1089,15 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
       Integer primaryBuckets) {
     HashMap localBucket2RegionMap = (HashMap)region_FPR.getDataStore()
         .getSizeLocally();
-    getLogWriter().info(
+    LogWriterSupport.getLogWriter().info(
         "Size of the " + region_FPR + " in this VM :- "
             + localBucket2RegionMap.size() + "List of buckets : "
             + localBucket2RegionMap.keySet());
     assertEquals(numBuckets.intValue(), localBucket2RegionMap.size());
-    getLogWriter().info(
+    LogWriterSupport.getLogWriter().info(
         "Size of primary buckets the " + region_FPR + " in this VM :- "
             + region_FPR.getDataStore().getNumberOfPrimaryBucketsManaged());
-    getLogWriter().info(
+    LogWriterSupport.getLogWriter().info(
         "Lit of Primaries in this VM :- "
             + region_FPR.getDataStore().getAllLocalPrimaryBucketIds());
     
@@ -1107,15 +1109,15 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
       Integer numBuckets, Integer primaryBuckets) {
     HashMap localBucket2RegionMap = (HashMap)region_FPR.getDataStore()
         .getSizeLocally();
-    getLogWriter().info(
+    LogWriterSupport.getLogWriter().info(
         "Size of the " + region_FPR + " in this VM :- "
             + localBucket2RegionMap.size() + "List of buckets : "
             + localBucket2RegionMap.keySet());
     assertEquals(numBuckets.intValue(), localBucket2RegionMap.size());
-    getLogWriter().info(
+    LogWriterSupport.getLogWriter().info(
         "Size of primary buckets the " + region_FPR + " in this VM :- "
             + region_FPR.getDataStore().getNumberOfPrimaryBucketsManaged());
-    getLogWriter().info(
+    LogWriterSupport.getLogWriter().info(
         "Lit of Primaries in this VM :- "
             + region_FPR.getDataStore().getAllLocalPrimaryBucketIds());
 
@@ -1304,8 +1306,8 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
                 return excuse;
               }
             };
-            DistributedTestCase.waitForCriterion(wc, 20000, 500, false);
-            getLogWriter().info("end of beforeCalculatingStartingBucketId");
+            Wait.waitForCriterion(wc, 20000, 500, false);
+            LogWriterSupport.getLogWriter().info("end of beforeCalculatingStartingBucketId");
           }
         });
   }
@@ -1362,7 +1364,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
             throw e;
           }
           catch (Throwable t) {
-            getLogWriter().error(t);
+            LogWriterSupport.getLogWriter().error(t);
           }
         }
       }
@@ -1376,7 +1378,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
         throw e;
       }
       catch (Throwable t) {
-        getLogWriter().error("Error in closing the cache ", t);
+        LogWriterSupport.getLogWriter().error("Error in closing the cache ", t);
         
       }
     }
@@ -1384,7 +1386,7 @@ public class FixedPartitioningTestBase extends DistributedTestCase {
     try {
       cleanDiskDirs();
     } catch(IOException e) {
-      getLogWriter().error("Error cleaning disk dirs", e);
+      LogWriterSupport.getLogWriter().error("Error cleaning disk dirs", e);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
index 46e54b6..8942c49 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
@@ -23,6 +23,7 @@ import com.gemstone.gemfire.cache.FixedPartitionAttributes;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Wait;
 
 public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
     FixedPartitioningTestBase {
@@ -492,7 +493,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               "Customer", "Order", "Shipment" });
 
       member3.invoke(FixedPartitioningTestBase.class, "closeCache");
-      pause(4000);
+      Wait.pause(4000);
 
       member1.invoke(FixedPartitioningTestBase.class,
           "checkPrimaryBucketsForColocationAfterCacheClosed", new Object[] {
@@ -522,7 +523,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               null, 2, 50, 20, new CustomerFixedPartitionResolver(), "Order",
               false });
 
-      pause(4000);
+      Wait.pause(4000);
 
       member1.invoke(FixedPartitioningTestBase.class,
           "validateAfterPutPartitionedRegion", new Object[] { "Customer",
@@ -553,7 +554,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               null, 2, 50, 20, new CustomerFixedPartitionResolver(), "Order",
               false });
 
-      pause(4000);
+      Wait.pause(4000);
 
       member1.invoke(FixedPartitioningTestBase.class,
           "validateAfterPutPartitionedRegion", new Object[] { "Customer",
@@ -687,7 +688,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               "Customer", "Order", "Shipment" });
 
       member3.invoke(FixedPartitioningTestBase.class, "closeCache");
-      pause(4000);
+      Wait.pause(4000);
 
       member1.invoke(FixedPartitioningTestBase.class,
           "checkPrimaryBucketsForColocationAfterCacheClosed", new Object[] {
@@ -717,7 +718,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               null, 2, 50, 20, new CustomerFixedPartitionResolver(), "Order",
               false });
 
-      pause(4000);
+      Wait.pause(4000);
 
       member1.invoke(FixedPartitioningTestBase.class,
           "validateAfterPutPartitionedRegion", new Object[] { "Customer",
@@ -748,7 +749,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               null, 2, 50, 20, new CustomerFixedPartitionResolver(), "Order",
               false });
 
-      pause(4000);
+      Wait.pause(4000);
 
       member1.invoke(FixedPartitioningTestBase.class,
           "validateAfterPutPartitionedRegion", new Object[] { "Customer",
@@ -802,7 +803,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
 
     member2.invoke(FixedPartitioningTestBase.class, "closeCache");
 
-    pause(1000);
+    Wait.pause(1000);
 
     member2.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
     fpa1 = FixedPartitionAttributes.createFixedPartition(Quarter2, true, 3);
@@ -910,14 +911,14 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
         "createRegionWithPartitionAttributes", new Object[] { "Quarter",
             fpaList, 1, 40, 12, new QuarterPartitionResolver(), null, true });
 
-    pause(4000);
+    Wait.pause(4000);
     member2.invoke(FixedPartitioningTestBase.class, "getForQuarter",
         new Object[] { "Quarter", Quarter1 });
     member2.invoke(FixedPartitioningTestBase.class, "getForQuarter",
         new Object[] { "Quarter", Quarter2 });
     member2.invoke(FixedPartitioningTestBase.class,
         "checkPrimaryDataPersistence", new Object[] { Quarter2 });
-    pause(2000);
+    Wait.pause(2000);
     member2.invoke(FixedPartitioningTestBase.class,
         "checkPrimaryBucketsForQuarter", new Object[] { 6, 6 });
 
@@ -931,7 +932,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
         "createRegionWithPartitionAttributes", new Object[] { "Quarter",
             fpaList, 1, 40, 12, new QuarterPartitionResolver(), null, true });
 
-    pause(4000);
+    Wait.pause(4000);
 
     member1.invoke(FixedPartitioningTestBase.class,
         "checkPrimaryDataPersistence", new Object[] { Quarter1 });
@@ -1065,7 +1066,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               null, 1, 50, 20, new CustomerFixedPartitionResolver(), "Order",
               false });
 
-      pause(4000);
+      Wait.pause(4000);
       member1.invoke(FixedPartitioningTestBase.class,
           "checkPrimaryBucketsForColocation", new Object[] { 10, 5, "Customer",
               "Order", "Shipment" });
@@ -1179,7 +1180,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
               fpaList, 1, 50, 20, new CustomerFixedPartitionResolver(), null,
               true });
 
-      pause(4000);
+      Wait.pause(4000);
       member2.invoke(FixedPartitioningTestBase.class, "getForColocation",
           new Object[] { "Customer", "Order", "Shipment" });
 
@@ -1248,7 +1249,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
         "createRegionWithPartitionAttributes", new Object[] { "Quarter",
             fpaList, 0, 40, 12, new QuarterPartitionResolver(), null, true });
 
-    pause(4000);
+    Wait.pause(4000);
     member2.invoke(FixedPartitioningTestBase.class,
         "checkPrimarySecondaryData", new Object[] { Quarter3, false });
 
@@ -1375,7 +1376,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
         "createRegionWithPartitionAttributes", new Object[] { "Quarter",
             fpaList, 1, 40, 12, new QuarterPartitionResolver(), null, true });
 
-    pause(4000);
+    Wait.pause(4000);
     member4.invoke(FixedPartitioningTestBase.class,
         "checkPrimarySecondaryData", new Object[] { Quarter4, false });
 
@@ -1425,7 +1426,7 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest extends
     member1.invoke(FixedPartitioningTestBase.class, "closeCache");
     member2.invoke(FixedPartitioningTestBase.class, "closeCache");
 
-    pause(1000);
+    Wait.pause(1000);
 
     member2.invoke(FixedPartitioningTestBase.class, "createCacheOnMember");
     member2.invoke(FixedPartitioningTestBase.class,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRVVRecoveryDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRVVRecoveryDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRVVRecoveryDUnitTest.java
index f3c72d6..f620104 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRVVRecoveryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRVVRecoveryDUnitTest.java
@@ -67,10 +67,14 @@ import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.LogWriterSupport;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase {
   
@@ -83,7 +87,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
   @Override
   public void tearDown2() throws Exception {
     super.tearDown2();
-    invokeInEveryVM(PersistentRecoveryOrderDUnitTest.class, "resetAckWaitThreshold");
+    Invoke.invokeInEveryVM(PersistentRecoveryOrderDUnitTest.class, "resetAckWaitThreshold");
   }
   
   public void testNoConcurrencyChecks () {
@@ -216,7 +220,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
       }
       
     });
-    IgnoredException ex = IgnoredException.addExpectedException("DiskAccessException");
+    IgnoredException ex = IgnoredException.addIgnoredException("DiskAccessException");
     try {
       //Force expiration, with our test hook that should close the cache
       tombstoneService = cache.getTombstoneService();
@@ -286,7 +290,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
 
           // We should wait for timeout time so that tomstones are expired
           // right away when they are gIId based on their original timestamp.
-          pause((int) TEST_REPLICATED_TOMBSTONE_TIMEOUT);
+          Wait.pause((int) TEST_REPLICATED_TOMBSTONE_TIMEOUT);
 
           TombstoneService.REPLICATED_TOMBSTONE_TIMEOUT = TEST_REPLICATED_TOMBSTONE_TIMEOUT;
           TombstoneService.EXPIRED_TOMBSTONE_LIMIT = entryCount;
@@ -296,7 +300,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
           assertEquals(entryCount, getTombstoneCount(region));
 
           getCache().getLogger().fine("Waiting for maximumSleepTime ms");
-          pause(10000); // maximumSleepTime+500 in TombstoneSweeper GC thread
+          Wait.pause(10000); // maximumSleepTime+500 in TombstoneSweeper GC thread
 
           // Tombstones should have been expired and garbage collected by now by
           // TombstoneService.
@@ -391,7 +395,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
         Cache cache = getCache();
         Region region = cache.getRegion("prRegion");
         while (!region.get("testKey").equals("testValue2")) {
-          pause(100);
+          Wait.pause(100);
         }
         region.destroy("testKey");
       }
@@ -412,7 +416,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
         
         Region.Entry entry = ((PartitionedRegion)region).getEntry("testKey", true /*Entry is destroyed*/);
         RegionEntry re = ((EntrySnapshot)entry).getRegionEntry();
-        getLogWriter().fine("RegionEntry for testKey: " + re.getKey() + " " + re.getValueInVM((LocalRegion) region));
+        LogWriterSupport.getLogWriter().fine("RegionEntry for testKey: " + re.getKey() + " " + re.getValueInVM((LocalRegion) region));
         assertTrue(re.getValueInVM((LocalRegion) region) instanceof Tombstone);
         
         VersionTag tag = re.getVersionStamp().asVersionTag();
@@ -752,7 +756,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
     //and then do the wait in the flusher thread.
     
     //Setup the callbacks to wait for krf creation and throw an exception
-    IgnoredException ex = IgnoredException.addExpectedException("DiskAccessException");
+    IgnoredException ex = IgnoredException.addIgnoredException("DiskAccessException");
     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER=true;
     try {
       final CountDownLatch krfCreated = new CountDownLatch(1);
@@ -809,7 +813,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
 
       //Wait for the region to be destroyed. The region won't be destroyed
       //until the async flusher thread ends up switching oplogs
-      waitForCriterion(new WaitCriterion() {
+      Wait.waitForCriterion(new WaitCriterion() {
 
         @Override
         public boolean done() {
@@ -836,7 +840,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
     for(int i = 0; i < 3; i++) {
       NonTXEntry entry = (NonTXEntry) recoveredRegion.getEntry("key" + i);
       tagsFromKrf[i] = entry.getRegionEntry().getVersionStamp().asVersionTag();
-      getLogWriter().info("krfTag[" + i + "]="+ tagsFromKrf[i] + ",value=" + entry.getValue());
+      LogWriterSupport.getLogWriter().info("krfTag[" + i + "]="+ tagsFromKrf[i] + ",value=" + entry.getValue());
     }
     
     closeCache();
@@ -851,7 +855,7 @@ public class PersistentRVVRecoveryDUnitTest extends PersistentReplicatedTestBase
       for(int i = 0; i < 3; i++) {
         NonTXEntry entry = (NonTXEntry) recoveredRegion.getEntry("key" + i);
         tagsFromCrf[i] = entry.getRegionEntry().getVersionStamp().asVersionTag();
-        getLogWriter().info("crfTag[" + i + "]="+ tagsFromCrf[i] + ",value=" + entry.getValue());
+        LogWriterSupport.getLogWriter().info("crfTag[" + i + "]="+ tagsFromCrf[i] + ",value=" + entry.getValue());
       }
       
       //Make sure the version tags from the krf and the crf match.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
index 16f7c6d..0a7e64e 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
@@ -73,10 +73,13 @@ import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.LogWriterSupport;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * This is a test of how persistent distributed
@@ -108,31 +111,31 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     
     putAnEntry(vm0);
     
-    getLogWriter().info("closing region in vm0");
+    LogWriterSupport.getLogWriter().info("closing region in vm0");
     closeRegion(vm0);
     
     updateTheEntry(vm1);
     
-    getLogWriter().info("closing region in vm1");
+    LogWriterSupport.getLogWriter().info("closing region in vm1");
     closeRegion(vm1);
     
     
     //This ought to wait for VM1 to come back
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     AsyncInvocation future = createPersistentRegionAsync(vm0);
     
     waitForBlockedInitialization(vm0);
     
     assertTrue(future.isAlive());
     
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     
     future.join(MAX_WAIT);
@@ -161,9 +164,9 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     
     putAnEntry(vm0);
@@ -178,17 +181,17 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
       }
     });
     
-    getLogWriter().info("closing region in vm0");
+    LogWriterSupport.getLogWriter().info("closing region in vm0");
     closeRegion(vm0);
     
     updateTheEntry(vm1);
     
-    getLogWriter().info("closing region in vm1");
+    LogWriterSupport.getLogWriter().info("closing region in vm1");
     closeCache(vm1);
     
     
     //This ought to wait for VM1 to come back
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     AsyncInvocation future = createPersistentRegionAsync(vm0);
     
     waitForBlockedInitialization(vm0);
@@ -206,7 +209,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
           adminDS = AdminDistributedSystemFactory.getDistributedSystem(config);
           adminDS.connect();
           Set<PersistentID> missingIds = adminDS.getMissingPersistentMembers();
-          getLogWriter().info("waiting members=" + missingIds);
+          LogWriterSupport.getLogWriter().info("waiting members=" + missingIds);
           assertEquals(1, missingIds.size());
           PersistentID missingMember = missingIds.iterator().next();
           adminDS.revokePersistentMember(
@@ -249,8 +252,8 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     
     //Now, we should not be able to create a region
     //in vm1, because the this member was revoked
-    getLogWriter().info("Creating region in VM1");
-    IgnoredException e = IgnoredException.addExpectedException(RevokedPersistentDataException.class.getSimpleName(), vm1);
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
+    IgnoredException e = IgnoredException.addIgnoredException(RevokedPersistentDataException.class.getSimpleName(), vm1);
     try {
       createPersistentRegion(vm1);
       fail("We should have received a split distributed system exception");
@@ -296,9 +299,9 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     
     putAnEntry(vm0);
@@ -313,12 +316,12 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
       }
     });
     
-    getLogWriter().info("closing region in vm0");
+    LogWriterSupport.getLogWriter().info("closing region in vm0");
     closeRegion(vm0);
     
     updateTheEntry(vm1);
     
-    getLogWriter().info("closing region in vm1");
+    LogWriterSupport.getLogWriter().info("closing region in vm1");
     closeRegion(vm1);
     
     final File dirToRevoke = getDiskDirForVM(vm1);
@@ -344,7 +347,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     });
     
     //This shouldn't wait, because we revoked the member
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
     
     checkForRecoveryStat(vm0, true);
@@ -363,8 +366,8 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     
     //Now, we should not be able to create a region
     //in vm1, because the this member was revoked
-    getLogWriter().info("Creating region in VM1");
-    IgnoredException e = IgnoredException.addExpectedException(RevokedPersistentDataException.class.getSimpleName(), vm1);
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
+    IgnoredException e = IgnoredException.addIgnoredException(RevokedPersistentDataException.class.getSimpleName(), vm1);
     try {
       createPersistentRegion(vm1);
       fail("We should have received a split distributed system exception");
@@ -389,9 +392,9 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     createPersistentRegion(vm2);
     
@@ -407,28 +410,28 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
       }
     });
     
-    getLogWriter().info("closing region in vm0");
+    LogWriterSupport.getLogWriter().info("closing region in vm0");
     closeRegion(vm0);
     
     updateTheEntry(vm1);
     
-    getLogWriter().info("closing region in vm1");
+    LogWriterSupport.getLogWriter().info("closing region in vm1");
     closeRegion(vm1);
     
     updateTheEntry(vm2, "D");
     
-    getLogWriter().info("closing region in vm2");
+    LogWriterSupport.getLogWriter().info("closing region in vm2");
     closeRegion(vm2);
     
     
     //These ought to wait for VM2 to come back
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     AsyncInvocation future0 = createPersistentRegionAsync(vm0);
     
     waitForBlockedInitialization(vm0);
     assertTrue(future0.isAlive());
     
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     final AsyncInvocation future1 = createPersistentRegionAsync(vm1);
     waitForBlockedInitialization(vm1);
     assertTrue(future1.isAlive());
@@ -444,7 +447,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
           adminDS = AdminDistributedSystemFactory.getDistributedSystem(config);
           adminDS.connect();
           Set<PersistentID> missingIds = adminDS.getMissingPersistentMembers();
-          getLogWriter().info("waiting members=" + missingIds);
+          LogWriterSupport.getLogWriter().info("waiting members=" + missingIds);
           assertEquals(1, missingIds.size());
         } catch (AdminException e) {
           throw new RuntimeException(e);
@@ -463,7 +466,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
       }
     });
     
-    waitForCriterion(new WaitCriterion() {
+    Wait.waitForCriterion(new WaitCriterion() {
       
       public boolean done() {
         return !future1.isAlive();
@@ -486,7 +489,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
           adminDS = AdminDistributedSystemFactory.getDistributedSystem(config);
           adminDS.connect();
           final AdminDistributedSystem connectedDS = adminDS;
-          waitForCriterion(new WaitCriterion() {
+          Wait.waitForCriterion(new WaitCriterion() {
 
             public String description() {
               return "Waiting for waiting members to have 2 members";
@@ -579,12 +582,12 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     restoreBackup(vm1);
     
   //This ought to wait for VM1 to come back
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     AsyncInvocation future = createPersistentRegionAsync(vm0);
     waitForBlockedInitialization(vm0);
     assertTrue(future.isAlive());
     
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     
     future.join(MAX_WAIT);
@@ -1047,7 +1050,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     //so it will start up.
     createPersistentRegion(vm0);
 
-    IgnoredException e = IgnoredException.addExpectedException(ConflictingPersistentDataException.class.getSimpleName(), vm1);
+    IgnoredException e = IgnoredException.addIgnoredException(ConflictingPersistentDataException.class.getSimpleName(), vm1);
     try {
       //VM1 should not start up, because we should detect that vm1
       //was never in the same distributed system as vm0
@@ -1074,24 +1077,24 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     
     putAnEntry(vm0);
     
-    getLogWriter().info("closing region in vm0");
+    LogWriterSupport.getLogWriter().info("closing region in vm0");
     closeRegion(vm0);
     
     updateTheEntry(vm1);
     
-    getLogWriter().info("closing region in vm1");
+    LogWriterSupport.getLogWriter().info("closing region in vm1");
     closeRegion(vm1);
     
     
     //This ought to wait for VM1 to come back
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     AsyncInvocation future = createPersistentRegionAsync(vm0);
     
     waitForBlockedInitialization(vm0);
@@ -1177,7 +1180,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     final VM vm1 = host.getVM(1);
     final VM vm2 = host.getVM(2);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
     
     //Add a hook which will disconnect from the distributed
@@ -1242,7 +1245,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
       
       public void run() {
        final  Cache cache = getCache();
-        waitForCriterion(new WaitCriterion() {
+        Wait.waitForCriterion(new WaitCriterion() {
 
           public String description() {
             return "Waiting for creation of region " + REGION_NAME;
@@ -1343,7 +1346,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     
     createNonPersistentRegion(vm0);
     
-    IgnoredException e = IgnoredException.addExpectedException(IllegalStateException.class.getSimpleName(), vm1);
+    IgnoredException e = IgnoredException.addIgnoredException(IllegalStateException.class.getSimpleName(), vm1);
     try {
       createPersistentRegion(vm1);
       fail("Should have received an IllegalState exception");
@@ -1383,10 +1386,10 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
              Cache cache = getCache();
              Region region = cache.getRegion(REGION_NAME);
              if (region == null) {
-               getLogWriter().severe("removing listener for PersistentRecoveryOrderDUnitTest because region was not found: " + REGION_NAME);
+               LogWriterSupport.getLogWriter().severe("removing listener for PersistentRecoveryOrderDUnitTest because region was not found: " + REGION_NAME);
                Object old = DistributionMessageObserver.setInstance(null);
                if (old != this) {
-                 getLogWriter().severe("removed listener was not the invoked listener", new Exception("stack trace"));
+                 LogWriterSupport.getLogWriter().severe("removed listener was not the invoked listener", new Exception("stack trace"));
                }
                return;
              }
@@ -1670,18 +1673,18 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     
-    getLogWriter().info("Creating region in VM0");
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
     createPersistentRegion(vm0);
     putAnEntry(vm0);
-    getLogWriter().info("closing region in vm0");
+    LogWriterSupport.getLogWriter().info("closing region in vm0");
     closeCache(vm0);
     
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     createPersistentRegion(vm1);
     putAnEntry(vm1);
     
-    getLogWriter().info("Creating region in VM0");
-    IgnoredException ex = IgnoredException.addExpectedException("ConflictingPersistentDataException", vm0);
+    LogWriterSupport.getLogWriter().info("Creating region in VM0");
+    IgnoredException ex = IgnoredException.addIgnoredException("ConflictingPersistentDataException", vm0);
     try {
       //this should cause a conflict
       createPersistentRegion(vm0);
@@ -1694,7 +1697,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
       ex.remove();
     }
     
-    getLogWriter().info("closing region in vm1");
+    LogWriterSupport.getLogWriter().info("closing region in vm1");
     closeCache(vm1);
     
     //This should work now
@@ -1702,9 +1705,9 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     
     updateTheEntry(vm0);
     
-    ex = IgnoredException.addExpectedException("ConflictingPersistentDataException", vm1);
+    ex = IgnoredException.addIgnoredException("ConflictingPersistentDataException", vm1);
     //Now make sure vm1 gets a conflict
-    getLogWriter().info("Creating region in VM1");
+    LogWriterSupport.getLogWriter().info("Creating region in VM1");
     try {
       //this should cause a conflict
       createPersistentRegion(vm1);
@@ -1777,11 +1780,11 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
 
   @Override
   public Properties getDistributedSystemProperties() {
-    getLogWriter().info("Looking for ack-wait-threshold");
+    LogWriterSupport.getLogWriter().info("Looking for ack-wait-threshold");
     String s = System.getProperty("gemfire.ack-wait-threshold");
     if (s != null) {
       SAVED_ACK_WAIT_THRESHOLD = s;
-      getLogWriter().info("removing system property gemfire.ack-wait-threshold");
+      LogWriterSupport.getLogWriter().info("removing system property gemfire.ack-wait-threshold");
       System.getProperties().remove("gemfire.ack-wait-threshold");
     }
     Properties props = super.getDistributedSystemProperties();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentReplicatedTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentReplicatedTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentReplicatedTestBase.java
index ff82082..26baa6e 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentReplicatedTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentReplicatedTestBase.java
@@ -37,8 +37,11 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
 import com.gemstone.gemfire.internal.cache.RegionFactoryImpl;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 public abstract class PersistentReplicatedTestBase extends CacheTestCase {
 
@@ -54,7 +57,7 @@ public abstract class PersistentReplicatedTestBase extends CacheTestCase {
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    invokeInEveryVM(PersistentReplicatedTestBase.class,"setRegionName", new Object[]{getUniqueName()});
+    Invoke.invokeInEveryVM(PersistentReplicatedTestBase.class,"setRegionName", new Object[]{getUniqueName()});
     setRegionName(getUniqueName());
     diskDir = new File("diskDir-" + getName()).getAbsoluteFile();
     com.gemstone.gemfire.internal.FileUtil.delete(diskDir);
@@ -76,7 +79,7 @@ public abstract class PersistentReplicatedTestBase extends CacheTestCase {
     vm.invoke(new SerializableRunnable() {
   
       public void run() {
-        waitForCriterion(new WaitCriterion() {
+        Wait.waitForCriterion(new WaitCriterion() {
   
           public String description() {
             return "Waiting for another persistent member to come online";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/Bug40396DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/Bug40396DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/Bug40396DUnitTest.java
index ed85295..d96c26a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/Bug40396DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/Bug40396DUnitTest.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.DeltaEOFException;
 import com.gemstone.gemfire.internal.cache.tier.sockets.FaultyDelta;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 /**
@@ -208,6 +209,6 @@ public class Bug40396DUnitTest extends DistributedTestCase {
     server.invoke(Bug40396DUnitTest.class, "closeCache");
     server2.invoke(Bug40396DUnitTest.class, "closeCache");
     cache = null;
-    invokeInEveryVM(new SerializableRunnable() { public void run() { cache = null; } });
+    Invoke.invokeInEveryVM(new SerializableRunnable() { public void run() { cache = null; } });
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
index 96ba65c..da194e0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.NetworkSupport;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.ConnectionFactoryImpl;
@@ -169,7 +170,7 @@ public class BackwardCompatibilityHigherVersionClientDUnitTest extends
         "setHandshakeVersionForTesting");
     client1.invoke(BackwardCompatibilityHigherVersionClientDUnitTest.class,
         "createClientCache", new Object[] {
-            getServerHostName(server1.getHost()), port1 });
+            NetworkSupport.getServerHostName(server1.getHost()), port1 });
     client1.invoke(BackwardCompatibilityHigherVersionClientDUnitTest.class,
         "verifyConnectionToServerFailed");
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/76e324b5/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
index a2caaea..6ab92c2 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
@@ -37,7 +37,10 @@ import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.NetworkSupport;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * The Region Destroy Operation from Cache Client does not pass the Client side
@@ -105,10 +108,10 @@ public class Bug36269DUnitTest extends DistributedTestCase
   {
     try {
       createClientCache();
-      acquireConnectionsAndDestroyRegion(getServerHostName(Host.getHost(0)));
+      acquireConnectionsAndDestroyRegion(NetworkSupport.getServerHostName(Host.getHost(0)));
       server1.invoke(Bug36269DUnitTest.class, "verifyRegionDestroy");
       server2.invoke(Bug36269DUnitTest.class, "verifyRegionDestroy");
-      pause(5000);
+      Wait.pause(5000);
       verifyNoRegionDestroyOnOriginator();
     }
     catch (Exception ex) {
@@ -138,7 +141,7 @@ public class Bug36269DUnitTest extends DistributedTestCase
     new Bug36269DUnitTest("temp").createCache(props);
     CacheServerTestUtil.disableShufflingOfEndpoints();
     PoolImpl p;
-    String host = getServerHostName(Host.getHost(0));
+    String host = NetworkSupport.getServerHostName(Host.getHost(0));
     try {
       p = (PoolImpl)PoolManager.createFactory()
         .addServer(host, PORT1)
@@ -200,7 +203,7 @@ public class Bug36269DUnitTest extends DistributedTestCase
           return null;
         }
       };
-      DistributedTestCase.waitForCriterion(ev, 40 * 1000, 200, true);
+      Wait.waitForCriterion(ev, 40 * 1000, 200, true);
     }
     catch (Exception ex) {
       Assert.fail("failed while verifyRegionDestroy", ex);