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/03/01 01:09:46 UTC

[1/7] incubator-geode git commit: GEODE-774

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-949-2 adab5324a -> 527c43225


GEODE-774

Add and enhance junit tests for previous fix.


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

Branch: refs/heads/feature/GEODE-949-2
Commit: 1e31fa646f414aeea28c7fa28096b70dc2ccd120
Parents: 64855a2
Author: zhouxh <gz...@pivotal.io>
Authored: Fri Feb 26 16:21:04 2016 -0800
Committer: zhouxh <gz...@pivotal.io>
Committed: Mon Feb 29 12:37:09 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/BucketRegion.java    |  25 ++-
 .../internal/cache/DistributedRegion.java       |   2 +-
 .../AbstractDistributedRegionJUnitTest.java     | 166 +++++++++++++++++
 .../internal/cache/BucketRegionJUnitTest.java   | 186 +++++--------------
 .../cache/DistributedRegionJUnitTest.java       | 101 ++++++++++
 .../com/gemstone/gemfire/test/fake/Fakes.java   |   5 +-
 6 files changed, 339 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e31fa64/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
index 69f61c4..5b17040 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
@@ -922,13 +922,7 @@ implements Bucket
           logger.trace(LogMarker.DM, "LR.basicInvalidate: this cache has already seen this event {}", event);
         }
         if (!getConcurrencyChecksEnabled() || event.hasValidVersionTag()) {
-          if (!event.isOriginRemote()
-              && getBucketAdvisor().isPrimary()) {
-            // This cache has processed the event, forward operation
-            // and event messages to backup buckets
-            new InvalidateOperation(event).distribute();
-          }
-          event.invokeCallbacks(this,true, false);
+          distributeInvalidateOperation(event);
         }
         return;
       }
@@ -936,6 +930,17 @@ implements Bucket
       endLocalWrite(event);
     }
   }
+
+  protected void distributeInvalidateOperation(EntryEventImpl event) {
+    if (!event.isOriginRemote()
+        && getBucketAdvisor().isPrimary()) {
+      // This cache has processed the event, forward operation
+      // and event messages to backup buckets
+      new InvalidateOperation(event).distribute();
+    }
+    event.invokeCallbacks(this,true, false);
+  }
+  
   @Override
   void basicInvalidatePart2(final RegionEntry re, final EntryEventImpl event,
       boolean clearConflict, boolean invokeCallbacks)
@@ -1312,7 +1317,7 @@ implements Bucket
         // This cache has processed the event, forward operation
         // and event messages to backup buckets
     	if (!getConcurrencyChecksEnabled() || event.hasValidVersionTag()) {
-          new UpdateEntryVersionOperation(event).distribute();
+          distributeUpdateEntryVersionOperation(event);
     	}
       }
       return;
@@ -1321,6 +1326,10 @@ implements Bucket
     }
   }
 
+  protected void distributeUpdateEntryVersionOperation(EntryEventImpl event) {
+    new UpdateEntryVersionOperation(event).distribute();
+  }
+  
   public int getRedundancyLevel()
   {
     return this.redundancy;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e31fa64/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
index 021eba6..19496da 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -2057,7 +2057,7 @@ public class DistributedRegion extends LocalRegion implements
     }
   }
 
-  private void distributeUpdateEntryVersion(EntryEventImpl event) {
+  protected void distributeUpdateEntryVersion(EntryEventImpl event) {
     if (!this.regionInvalid && event.isDistributed() && !event.isOriginRemote()
         && !isTX() /* only distribute if non-tx */) {
       if (event.isDistributed() && !event.isOriginRemote()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e31fa64/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
new file mode 100755
index 0000000..8eaeeb5
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/AbstractDistributedRegionJUnitTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.internal.cache;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.Test;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.fake.Fakes;
+
+import junit.framework.TestCase;
+
+public abstract class AbstractDistributedRegionJUnitTest extends TestCase {
+  protected static final Logger logger = LogService.getLogger();
+  
+  private RegionAttributes createRegionAttributes(
+      boolean isConcurrencyChecksEnabled) {
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(DataPolicy.REPLICATE);
+    factory.setConcurrencyChecksEnabled(isConcurrencyChecksEnabled); //
+    RegionAttributes ra = factory.create();
+    return ra;
+  }
+  
+  private EventID createDummyEventID() {
+    byte[] memId = { 1,2,3 };
+    EventID eventId = new EventID(memId, 11, 12, 13);
+    return eventId;
+  }
+
+  private EntryEventImpl createDummyEvent(DistributedRegion region) {
+    // create a dummy event id
+    EventID eventId = createDummyEventID();
+    String key = "key1";
+    String value = "Value1";
+
+    // create an event
+    EntryEventImpl event = EntryEventImpl.create(region, Operation.CREATE, key,
+        value, null,  false /* origin remote */, null,
+        false /* generateCallbacks */,
+        eventId);
+    // avoid calling invokeCallbacks
+    event.callbacksInvoked(true);
+
+    return event;
+  }
+  
+  private VersionTag createVersionTag(boolean validVersionTag) {
+    InternalDistributedMember remotemember = mock(InternalDistributedMember.class);
+    VersionTag tag = VersionTag.create(remotemember);
+    if (validVersionTag) {
+      tag.setRegionVersion(1);
+      tag.setEntryVersion(1);
+    }
+    return tag;
+  }
+
+  private void doTest(DistributedRegion region, EntryEventImpl event, int cnt) {
+    // do the virtualPut test
+    verifyDistributeUpdate(region, event, cnt);
+    
+    // do the basicDestroy test
+    verifyDistributeDestroy(region, event, cnt);
+    
+    // do the basicInvalidate test
+    verifyDistributeInvalidate(region, event, cnt);
+    
+    // do the basicUpdateEntryVersion test
+    verifyDistributeUpdateEntryVersion(region, event, cnt);
+  }
+  
+  protected abstract void setInternalRegionArguments(InternalRegionArguments ira);
+  protected abstract DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,RegionAttributes ra, 
+      InternalRegionArguments ira, GemFireCacheImpl cache);
+  protected abstract void verifyDistributeUpdate(DistributedRegion region, EntryEventImpl event, int cnt);
+  protected abstract void verifyDistributeDestroy(DistributedRegion region, EntryEventImpl event, int cnt);
+  protected abstract void verifyDistributeInvalidate(DistributedRegion region, EntryEventImpl event, int cnt);
+  protected abstract void verifyDistributeUpdateEntryVersion(DistributedRegion region, EntryEventImpl event, int cnt);
+
+  protected DistributedRegion prepare(boolean isConcurrencyChecksEnabled) {
+    GemFireCacheImpl cache = Fakes.cache();
+    
+    // create region attributes and internal region arguments
+    RegionAttributes ra = createRegionAttributes(isConcurrencyChecksEnabled);
+    InternalRegionArguments ira = new InternalRegionArguments();
+    
+    setInternalRegionArguments(ira);
+    
+    // create a region object
+    DistributedRegion region = createAndDefineRegion(isConcurrencyChecksEnabled, ra, ira, cache);
+    if (isConcurrencyChecksEnabled) {
+      region.enableConcurrencyChecks();
+    }
+    
+    doNothing().when(region).notifyGatewaySender(any(), any());
+    doReturn(true).when(region).hasSeenEvent(any(EntryEventImpl.class));
+    return region;
+  }
+  
+  @Test
+  public void testConcurrencyFalseTagNull() {
+    // case 1: concurrencyCheckEanbled = false, version tag is null: distribute
+    DistributedRegion region = prepare(false);
+    EntryEventImpl event = createDummyEvent(region);
+    assertNull(event.getVersionTag());
+    doTest(region, event, 1);
+  }
+
+  @Test
+  public void testConcurrencyTrueTagNull() {
+    // case 2: concurrencyCheckEanbled = true,  version tag is null: not to distribute
+    DistributedRegion region = prepare(true);
+    EntryEventImpl event = createDummyEvent(region);
+    assertNull(event.getVersionTag());
+    doTest(region, event, 0);
+  }
+  
+  @Test
+  public void testConcurrencyTrueTagInvalid() {
+    // case 3: concurrencyCheckEanbled = true,  version tag is invalid: not to distribute
+    DistributedRegion region = prepare(true);
+    EntryEventImpl event = createDummyEvent(region);
+    VersionTag tag = createVersionTag(false);
+    event.setVersionTag(tag);
+    assertFalse(tag.hasValidVersion());
+    doTest(region, event, 0);
+  }
+    
+  @Test
+  public void testConcurrencyTrueTagValid() {
+    // case 4: concurrencyCheckEanbled = true,  version tag is valid: distribute
+    DistributedRegion region = prepare(true);
+    EntryEventImpl event = createDummyEvent(region);
+    VersionTag tag = createVersionTag(true);
+    event.setVersionTag(tag);
+    assertTrue(tag.hasValidVersion());
+    doTest(region, event, 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e31fa64/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
index b2623fb..e8e4d1f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/BucketRegionJUnitTest.java
@@ -16,30 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache;
 
-import org.apache.logging.log4j.Logger;
-import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.distributed.internal.DSClock;
-import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.internal.cache.versions.VersionTag;
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import junit.framework.TestCase;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.*;
 
-import java.util.HashSet;
-import java.util.Properties;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -49,105 +35,44 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
  *
  */
 @Category(UnitTest.class)
-public class BucketRegionJUnitTest extends TestCase {
+public class BucketRegionJUnitTest extends DistributedRegionJUnitTest {
 
-  private static final Logger logger = LogService.getLogger();
-  GemFireCacheImpl cache = null;
-  InternalDistributedMember member = null;
-  PartitionedRegion pr = null;
-  BucketAdvisor ba = null;
-  RegionAttributes ra = null;
-  InternalRegionArguments ira = null;
-  BucketRegion br = null;
-  
-  private EntryEventImpl prepare(boolean isConcurrencyChecksEnabled) {
-    // mock cache, distributed system, distribution manager, and distributed member
-    cache = mock(GemFireCacheImpl.class);
-    InternalDistributedSystem ids = mock(InternalDistributedSystem.class);
-    DistributionManager dm = mock(DistributionManager.class);
-    member = mock(InternalDistributedMember.class);
-    InvalidateOperation io = mock(InvalidateOperation.class);
-
-    // mock PR, bucket advisor, DSClock 
-    pr = mock(PartitionedRegion.class);
-    ba = mock(BucketAdvisor.class);
-    DSClock clock = mock(DSClock.class);
-
-    // create some real objects
-    DistributionConfigImpl config = new DistributionConfigImpl(new Properties());
+  protected void setInternalRegionArguments(InternalRegionArguments ira) {
+    // PR specific
+    PartitionedRegion pr = mock(PartitionedRegion.class);
+    BucketAdvisor ba = mock(BucketAdvisor.class);
     ReadWriteLock primaryMoveLock = new ReentrantReadWriteLock();
     Lock activeWriteLock = primaryMoveLock.readLock();
-    GemFireCacheImpl.Stopper stopper = mock(GemFireCacheImpl.Stopper.class);
-
-    // define the mock objects' behaviors
-    when(cache.getDistributedSystem()).thenReturn(ids);
-    when(ids.getClock()).thenReturn(clock);
-    when(ids.getDistributionManager()).thenReturn(dm);
-    when(ids.getDistributedMember()).thenReturn(member);
-    when(dm.getConfig()).thenReturn(config);
-    when(member.getRoles()).thenReturn(new HashSet());
     when(ba.getActiveWriteLock()).thenReturn(activeWriteLock);
-    when (cache.getCancelCriterion()).thenReturn(stopper);
-    when(ids.getCancelCriterion()).thenReturn(stopper);
-
-    // create region attributes and internal region arguments
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    factory.setConcurrencyChecksEnabled(isConcurrencyChecksEnabled); //
-    ra = factory.create();
-    ira = new InternalRegionArguments().setPartitionedRegion(pr)
-        .setPartitionedRegionBucketRedundancy(1)
-        .setBucketAdvisor(ba);
-    // create a bucket region object
-    br = new BucketRegion("testRegion", ra, null, cache, ira);
+    when(ba.isPrimary()).thenReturn(true);
     
+    ira.setPartitionedRegion(pr)
+      .setPartitionedRegionBucketRedundancy(1)
+      .setBucketAdvisor(ba);
+  }
+  
+  protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
+      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
+    BucketRegion br = new BucketRegion("testRegion", ra, null, cache, ira);
+
     // since br is a real bucket region object, we need to tell mockito to monitor it
     br = Mockito.spy(br);
 
+//    doNothing().when(dm).addMembershipListener(any());
     doNothing().when(br).distributeUpdateOperation(any(), anyLong());
     doNothing().when(br).distributeDestroyOperation(any());
+    doNothing().when(br).distributeInvalidateOperation(any());
+    doNothing().when(br).distributeUpdateEntryVersionOperation(any());
     doNothing().when(br).checkForPrimary();
     doNothing().when(br).handleWANEvent(any());
     doReturn(false).when(br).needWriteLock(any());
-    doReturn(true).when(br).hasSeenEvent(any(EntryEventImpl.class));
     
-    EntryEventImpl event = createDummyEvent();
-    return event;
-  }
-  
-  private EventID createDummyEventID() {
-    byte[] memId = { 1,2,3 };
-    EventID eventId = new EventID(memId, 11, 12, 13);
-    return eventId;
+    return br;
   }
 
-  private EntryEventImpl createDummyEvent() {
-    // create a dummy event id
-    EventID eventId = createDummyEventID();
-    String key = "key1";
-    String value = "Value1";
-
-    // create an event
-    EntryEventImpl event = EntryEventImpl.create(br, Operation.CREATE, key,
-        value, null,  false /* origin remote */, null,
-        false /* generateCallbacks */,
-        eventId);
-
-    return event;
-  }
-  
-  private VersionTag createVersionTag(boolean invalid) {
-    VersionTag tag = VersionTag.create(member);
-    if (invalid == false) {
-      tag.setRegionVersion(1);
-      tag.setEntryVersion(1);
-    }
-    return tag;
-  }
-  
-  private void doTest(EntryEventImpl event, int cnt) {
-    // do the virtualPut test
+  protected void verifyDistributeUpdate(DistributedRegion region, EntryEventImpl event, int cnt) {
+    assertTrue(region instanceof BucketRegion);
+    BucketRegion br = (BucketRegion)region;
     br.virtualPut(event, false, false, null, false, 12345L, false);
     // verify the result
     if (cnt > 0) {
@@ -155,8 +80,11 @@ public class BucketRegionJUnitTest extends TestCase {
     } else {
       verify(br, never()).distributeUpdateOperation(eq(event), eq(12345L));
     }
-    
-    // do the basicDestroy test
+  }
+  
+  protected void verifyDistributeDestroy(DistributedRegion region, EntryEventImpl event, int cnt) {
+    assertTrue(region instanceof BucketRegion);
+    BucketRegion br = (BucketRegion)region;
     br.basicDestroy(event, false, null);
     // verify the result
     if (cnt > 0) {
@@ -164,45 +92,31 @@ public class BucketRegionJUnitTest extends TestCase {
     } else {
       verify(br, never()).distributeDestroyOperation(eq(event));
     }
-    
   }
   
-  @Test
-  public void testConcurrencyFalseTagNull() {
-    // case 1: concurrencyCheckEanbled = false, version tag is null: distribute
-    EntryEventImpl event = prepare(false);
-    assertNull(event.getVersionTag());
-    doTest(event, 1);
-  }
-
-  @Test
-  public void testConcurrencyTrueTagNull() {
-    // case 2: concurrencyCheckEanbled = true,  version tag is null: not to distribute
-    EntryEventImpl event = prepare(true);
-    assertNull(event.getVersionTag());
-    doTest(event, 0);
-  }
-  
-  @Test
-  public void testConcurrencyTrueTagInvalid() {
-    // case 3: concurrencyCheckEanbled = true,  version tag is invalid: not to distribute
-    EntryEventImpl event = prepare(true);
-    VersionTag tag = createVersionTag(true);
-    event.setVersionTag(tag);
-    assertFalse(tag.hasValidVersion());
-    doTest(event, 0);
+  protected void verifyDistributeInvalidate(DistributedRegion region, EntryEventImpl event, int cnt) {
+    assertTrue(region instanceof BucketRegion);
+    BucketRegion br = (BucketRegion)region;
+    br.basicInvalidate(event);
+    // verify the result
+    if (cnt > 0) {
+      verify(br, times(cnt)).distributeInvalidateOperation(eq(event));
+    } else {
+      verify(br, never()).distributeInvalidateOperation(eq(event));
+    }
   }
     
-  @Test
-  public void testConcurrencyTrueTagValid() {
-    // case 4: concurrencyCheckEanbled = true,  version tag is valid: distribute
-    EntryEventImpl event = prepare(true);
-    VersionTag tag = createVersionTag(false);
-    event.setVersionTag(tag);
-    assertTrue(tag.hasValidVersion());
-    doTest(event, 1);
+  protected void verifyDistributeUpdateEntryVersion(DistributedRegion region, EntryEventImpl event, int cnt) {
+    assertTrue(region instanceof BucketRegion);
+    BucketRegion br = (BucketRegion)region;
+    br.basicUpdateEntryVersion(event);
+    // verify the result
+    if (cnt > 0) {
+      verify(br, times(cnt)).distributeUpdateEntryVersionOperation(eq(event));
+    } else {
+      verify(br, never()).distributeUpdateEntryVersionOperation(eq(event));
+    }
   }
-
-
+  
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e31fa64/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
new file mode 100755
index 0000000..1383b76
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistributedRegionJUnitTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.internal.cache;
+
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import junit.framework.TestCase;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.*;
+
+/**
+ * @author gzhou
+ *
+ */
+@Category(UnitTest.class)
+public class DistributedRegionJUnitTest extends AbstractDistributedRegionJUnitTest {
+
+  protected void setInternalRegionArguments(InternalRegionArguments ira) {
+  }
+  
+  protected DistributedRegion createAndDefineRegion(boolean isConcurrencyChecksEnabled,
+      RegionAttributes ra, InternalRegionArguments ira, GemFireCacheImpl cache) {
+    DistributedRegion region = new DistributedRegion("testRegion", ra, null, cache, ira);
+    if (isConcurrencyChecksEnabled) {
+      region.enableConcurrencyChecks();
+    }
+    
+    // since it is a real region object, we need to tell mockito to monitor it
+    region = Mockito.spy(region);
+
+    doNothing().when(region).distributeUpdate(any(), anyLong(), anyBoolean(), anyBoolean(), any(), anyBoolean());
+    doNothing().when(region).distributeDestroy(any(), any());
+    doNothing().when(region).distributeInvalidate(any());
+    doNothing().when(region).distributeUpdateEntryVersion(any());
+    
+    return region;
+  }
+  
+  protected void verifyDistributeUpdate(DistributedRegion region, EntryEventImpl event, int cnt) {
+    region.virtualPut(event, false, false, null, false, 12345L, false);
+    // verify the result
+    if (cnt > 0) {
+      verify(region, times(cnt)).distributeUpdate(eq(event), eq(12345L), anyBoolean(), anyBoolean(), any(), anyBoolean());
+    } else {
+      verify(region, never()).distributeUpdate(eq(event), eq(12345L), anyBoolean(), anyBoolean(), any(), anyBoolean());
+    }
+  }
+
+  protected void verifyDistributeDestroy(DistributedRegion region, EntryEventImpl event, int cnt) {
+    region.basicDestroy(event, false, null);
+    // verify the result
+    if (cnt > 0) {
+      verify(region, times(cnt)).distributeDestroy(eq(event), any());
+    } else {
+      verify(region, never()).distributeDestroy(eq(event), any());
+    }
+  }
+
+  protected void verifyDistributeInvalidate(DistributedRegion region, EntryEventImpl event, int cnt) {
+    region.basicInvalidate(event);
+    // verify the result
+    if (cnt > 0) {
+      verify(region, times(cnt)).distributeInvalidate(eq(event));
+    } else {
+      verify(region, never()).distributeInvalidate(eq(event));
+    }
+  }
+
+  protected void verifyDistributeUpdateEntryVersion(DistributedRegion region, EntryEventImpl event, int cnt) {
+    region.basicUpdateEntryVersion(event);
+    // verify the result
+    if (cnt > 0) {
+      verify(region, times(cnt)).distributeUpdateEntryVersion(eq(event));
+    } else {
+      verify(region, never()).distributeUpdateEntryVersion(eq(event));
+    }
+  }
+  
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1e31fa64/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
index ffb4896..99644b7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
@@ -23,6 +23,7 @@ import java.net.UnknownHostException;
 import org.junit.Assert;
 
 import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.distributed.internal.DSClock;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -59,6 +60,7 @@ public class Fakes {
     DistributionConfig config = mock(DistributionConfig.class);
     DistributionManager distributionManager = mock(DistributionManager.class);
     CancelCriterion systemCancelCriterion = mock(CancelCriterion.class);
+    DSClock clock = mock(DSClock.class);
     
     InternalDistributedMember member;
     try {
@@ -77,7 +79,8 @@ public class Fakes {
     when(system.getConfig()).thenReturn(config);
     when(system.getDistributionManager()).thenReturn(distributionManager);
     when(system.getCancelCriterion()).thenReturn(systemCancelCriterion);
-    
+    when(system.getClock()).thenReturn(clock);
+
     when(distributionManager.getId()).thenReturn(member);
     when(distributionManager.getConfig()).thenReturn(config);
     when(distributionManager.getSystem()).thenReturn(system);


[7/7] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-949-2

Posted by kl...@apache.org.
Merge remote-tracking branch 'origin/develop' into feature/GEODE-949-2


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

Branch: refs/heads/feature/GEODE-949-2
Commit: 527c43225113466d3b2d067ab7a3037728ba022b
Parents: adab532 b4ed2aa
Author: Kirk Lund <kl...@pdx2-office-dhcp32.eng.vmware.com>
Authored: Mon Feb 29 16:09:50 2016 -0800
Committer: Kirk Lund <kl...@pdx2-office-dhcp32.eng.vmware.com>
Committed: Mon Feb 29 16:09:50 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/BucketRegion.java    |  25 +-
 .../internal/cache/DistributedRegion.java       |   2 +-
 .../AbstractDistributedRegionJUnitTest.java     | 166 ++++++++
 .../internal/cache/BucketRegionJUnitTest.java   | 186 +++------
 .../cache/DistributedRegionJUnitTest.java       | 101 +++++
 .../internal/process/PidFileJUnitTest.java      |   4 +-
 .../dunit/rules/DistributedDisconnectRule.java  |  44 +-
 .../rules/DistributedExternalResource.java      |  27 +-
 .../DistributedRestoreSystemProperties.java     |   5 +-
 .../gemfire/test/dunit/rules/RemoteInvoker.java |  10 +-
 .../com/gemstone/gemfire/test/fake/Fakes.java   |   5 +-
 geode-junit/build.gradle                        |   1 +
 .../gemfire/test/junit/ConditionalIgnore.java   |   1 -
 .../gemfire/test/junit/IgnoreCondition.java     |   1 -
 .../gemfire/test/junit/IgnoreUntil.java         |   1 -
 .../com/gemstone/gemfire/test/junit/Repeat.java |   3 +-
 .../com/gemstone/gemfire/test/junit/Retry.java  |   4 +-
 .../test/junit/categories/ContainerTest.java    |   3 +-
 .../test/junit/categories/DistributedTest.java  |   3 +-
 .../categories/DistributedTransactionsTest.java |   3 +-
 .../test/junit/categories/HydraTest.java        |   3 +-
 .../test/junit/categories/IntegrationTest.java  |   3 +-
 .../test/junit/categories/PerformanceTest.java  |   3 +-
 .../gemfire/test/junit/categories/UITest.java   |   3 +-
 .../gemfire/test/junit/categories/UnitTest.java |  13 +-
 .../gemfire/test/junit/categories/WanTest.java  |   5 +-
 .../test/junit/rules/ConditionalIgnoreRule.java |   1 -
 .../test/junit/rules/ExpectedTimeout.java       | 180 --------
 .../test/junit/rules/ExpectedTimeoutRule.java   |  42 +-
 .../test/junit/rules/IgnoreUntilRule.java       |   1 -
 .../gemfire/test/junit/rules/RepeatRule.java    |   3 +-
 .../gemfire/test/junit/rules/RetryRule.java     |   1 -
 .../gemfire/test/junit/rules/RuleList.java      |  95 +++++
 .../rules/SerializableExternalResource.java     | 107 -----
 .../test/junit/rules/SerializableRuleChain.java | 119 ------
 .../rules/SerializableTemporaryFolder.java      |  70 ----
 .../test/junit/rules/SerializableTestName.java  |  54 ---
 .../test/junit/rules/SerializableTestRule.java  |  33 --
 .../junit/rules/SerializableTestWatcher.java    |  29 --
 .../test/junit/rules/SerializableTimeout.java   | 119 ------
 .../serializable/FieldSerializationUtils.java   |  48 +++
 .../serializable/FieldsOfTemporaryFolder.java   |  26 ++
 .../rules/serializable/FieldsOfTestName.java    |  24 ++
 .../rules/serializable/FieldsOfTimeout.java     |  26 ++
 .../SerializableExternalResource.java           |  25 ++
 .../serializable/SerializableRuleList.java      |  78 ++++
 .../SerializableTemporaryFolder.java            |  70 ++++
 .../serializable/SerializableTestName.java      |  65 +++
 .../serializable/SerializableTestRule.java      |  28 ++
 .../serializable/SerializableTestWatcher.java   |  26 ++
 .../rules/serializable/SerializableTimeout.java | 104 +++++
 .../junit/support/DefaultIgnoreCondition.java   |   3 +-
 .../IgnoreConditionEvaluationException.java     |   1 -
 .../junit/rules/ExpectedTimeoutJUnitTest.java   | 204 ---------
 .../junit/rules/ExpectedTimeoutRuleTest.java    | 246 +++++++++++
 .../test/junit/rules/IgnoreUntilRuleTest.java   | 145 +++++++
 .../test/junit/rules/RepeatRuleTest.java        | 411 +++++++++++++++++++
 .../rules/RetryRuleGlobalWithErrorTest.java     | 326 +++++++++++++++
 .../rules/RetryRuleGlobalWithExceptionTest.java | 332 +++++++++++++++
 .../rules/RetryRuleLocalWithErrorTest.java      | 265 ++++++++++++
 .../rules/RetryRuleLocalWithExceptionTest.java  | 276 +++++++++++++
 .../gemfire/test/junit/rules/RuleListTest.java  | 209 ++++++++++
 .../gemfire/test/junit/rules/TestRunner.java    |  35 ++
 .../examples/RepeatingTestCasesExampleTest.java |  15 +-
 .../rules/examples/RetryRuleExampleTest.java    |  20 +-
 .../rules/examples/RuleAndClassRuleTest.java    | 147 +++++++
 .../SerializableExternalResourceTest.java       |  79 ++++
 .../serializable/SerializableRuleListTest.java  |  89 ++++
 .../SerializableTemporaryFolderTest.java        |  90 ++++
 .../serializable/SerializableTestNameTest.java  |  84 ++++
 .../SerializableTestWatcherTest.java            |  79 ++++
 .../serializable/SerializableTimeoutTest.java   | 106 +++++
 .../rules/tests/ExpectedTimeoutRuleTest.java    | 214 ----------
 .../junit/rules/tests/IgnoreUntilRuleTest.java  | 121 ------
 .../junit/rules/tests/JUnitRuleTestSuite.java   |  33 --
 .../test/junit/rules/tests/RepeatRuleTest.java  | 304 --------------
 .../tests/RetryRuleGlobalWithErrorTest.java     | 250 -----------
 .../tests/RetryRuleGlobalWithExceptionTest.java | 254 ------------
 .../tests/RetryRuleLocalWithErrorTest.java      | 207 ----------
 .../tests/RetryRuleLocalWithExceptionTest.java  | 213 ----------
 .../junit/rules/tests/RuleAndClassRuleTest.java | 138 -------
 .../test/junit/rules/tests/TestRunner.java      |  37 --
 82 files changed, 3962 insertions(+), 2970 deletions(-)
----------------------------------------------------------------------



[3/7] incubator-geode git commit: GEODE-953: Cleanup geode-junit

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RepeatingTestCasesExampleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RepeatingTestCasesExampleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RepeatingTestCasesExampleTest.java
index 5ee647b..7fdf696 100755
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RepeatingTestCasesExampleTest.java
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RepeatingTestCasesExampleTest.java
@@ -33,9 +33,8 @@ import com.gemstone.gemfire.test.junit.rules.RepeatRule;
 
 /**
  * The RepeatingTestCasesExampleTest class is a test suite of test cases testing the contract and functionality
- * of the JUnit @Repeat annotation on a test suite class test case methods.
+ * of the JUnit {@literal @}Repeat annotation on a test suite class test case methods.
  *
- * @author John Blum
  * @see org.junit.Test
  * @see com.gemstone.gemfire.test.junit.Repeat
  * @see com.gemstone.gemfire.test.junit.rules.RepeatRule
@@ -43,10 +42,10 @@ import com.gemstone.gemfire.test.junit.rules.RepeatRule;
 @Category(UnitTest.class)
 public class RepeatingTestCasesExampleTest {
 
-  private static AtomicInteger repeatOnceCounter = new AtomicInteger(0);
-  private static AtomicInteger repeatOnlyOnceCounter = new AtomicInteger(0);
-  private static AtomicInteger repeatTenTimesCounter = new AtomicInteger(0);
-  private static AtomicInteger repeatTwiceCounter = new AtomicInteger(0);
+  private static final AtomicInteger repeatOnceCounter = new AtomicInteger(0);
+  private static final AtomicInteger repeatOnlyOnceCounter = new AtomicInteger(0);
+  private static final AtomicInteger repeatTenTimesCounter = new AtomicInteger(0);
+  private static final AtomicInteger repeatTwiceCounter = new AtomicInteger(0);
 
   @Rule
   public RepeatRule repeatRule = new RepeatRule();
@@ -54,6 +53,10 @@ public class RepeatingTestCasesExampleTest {
   @BeforeClass
   public static void setupBeforeClass() {
     System.setProperty("tdd.example.test.case.two.repetitions", "2");
+    repeatOnceCounter.set(0);
+    repeatOnlyOnceCounter.set(0);
+    repeatTenTimesCounter.set(0);
+    repeatTwiceCounter.set(0);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RetryRuleExampleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RetryRuleExampleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RetryRuleExampleTest.java
index f6d70a2..3561169 100755
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RetryRuleExampleTest.java
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RetryRuleExampleTest.java
@@ -18,21 +18,29 @@ package com.gemstone.gemfire.test.junit.rules.examples;
 
 import static org.assertj.core.api.Assertions.*;
 
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.rules.RetryRule;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.RetryRule;
-
+/**
+ * Example usage of {@link RetryRule} with global scope.
+ */
 @Category(UnitTest.class)
 public class RetryRuleExampleTest {
 
-  @Rule
-  public final transient RetryRule retry = new RetryRule(2);
-  
   private static int count = 0;
 
+  @Rule
+  public RetryRule retry = new RetryRule(2);
+
+  @BeforeClass
+  public static void beforeClass() {
+    count = 0;
+  }
+
   @Test
   public void unreliableTestWithRaceConditions() {
     count++;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RuleAndClassRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RuleAndClassRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RuleAndClassRuleTest.java
new file mode 100755
index 0000000..4e69ec3
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/examples/RuleAndClassRuleTest.java
@@ -0,0 +1,147 @@
+/*
+ * 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.junit.rules.examples;
+
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.rules.TestRunner;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runner.Result;
+import org.junit.runners.model.Statement;
+
+/**
+ * Example usage of a rule as both a method {@literal @}Rule and a {@literal @}ClassRule.
+ */
+@Category(UnitTest.class)
+public class RuleAndClassRuleTest {
+
+  @Test
+  public void usingRuleAsRuleAndClassRuleShouldInvokeBeforeClass() {
+    Result result = TestRunner.runTest(UsingRuleAsRuleAndClassRule.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(UsingRuleAsRuleAndClassRule.staticRule.beforeClassInvoked).isEqualTo(true);
+  }
+  
+  @Test
+  public void usingRuleAsRuleAndClassRuleShouldInvokeAfterClass() {
+    Result result = TestRunner.runTest(UsingRuleAsRuleAndClassRule.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(UsingRuleAsRuleAndClassRule.staticRule.afterClassInvoked).isEqualTo(true);
+  }
+
+  @Test
+  public void usingRuleAsRuleAndClassRuleShouldInvokeBefore() {
+    Result result = TestRunner.runTest(UsingRuleAsRuleAndClassRule.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(UsingRuleAsRuleAndClassRule.staticRule.beforeInvoked).isEqualTo(true);
+  }
+
+  @Test
+  public void usingRuleAsRuleAndClassRuleShouldInvokeAfter() {
+    Result result = TestRunner.runTest(UsingRuleAsRuleAndClassRule.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(UsingRuleAsRuleAndClassRule.staticRule.afterInvoked).isEqualTo(true);
+  }
+
+  /**
+   * Implementation of TestRule that records the callbacks invoked on it. Used
+   * by {@link UsingRuleAsRuleAndClassRule}.
+   */
+  public static class SpyRule implements TestRule {
+    boolean beforeClassInvoked = false;
+    boolean afterClassInvoked = false;
+    boolean beforeInvoked = false;
+    boolean afterInvoked = false;
+    
+    @Override
+    public Statement apply(final Statement base, final Description description) {
+      if (description.isTest()) {
+        return statement(base);
+      } else if (description.isSuite()) {
+        return statementClass(base);
+      }
+      return base;
+    }
+
+    private Statement statement(final Statement base) {
+      return new Statement() {
+        @Override
+        public void evaluate() throws Throwable {
+          before();
+          try {
+            base.evaluate();
+          } finally {
+            after();
+          }
+        }
+      };
+    }
+    
+    private Statement statementClass(final Statement base) {
+      return new Statement() {
+        @Override
+        public void evaluate() throws Throwable {
+          beforeClass();
+          try {
+            base.evaluate();
+          } finally {
+            afterClass();
+          }
+        }
+      };
+    }
+    
+    private void beforeClass() {
+      this.beforeClassInvoked = true;
+    }
+    
+    private void afterClass() {
+      this.afterClassInvoked = true;
+    }
+    
+    private void before() {
+      this.beforeInvoked = true;
+    }
+    
+    private void after() {
+      this.afterInvoked = true;
+    }
+  }
+
+  /**
+   * Used by the tests in {@link RuleAndClassRuleTest}.
+   */
+  public static class UsingRuleAsRuleAndClassRule {
+    @ClassRule
+    public static SpyRule staticRule = new SpyRule();
+    @Rule
+    public SpyRule rule = staticRule;
+    @Test
+    public void doTest() throws Exception {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResourceTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResourceTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResourceTest.java
new file mode 100755
index 0000000..aa7ea40
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResourceTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExternalResource;
+
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/**
+ * Unit tests for {@link SerializableExternalResource}.
+ */
+@Category(UnitTest.class)
+public class SerializableExternalResourceTest {
+
+  @Test
+  public void hasZeroFields() throws Exception {
+    Field[] fields = ExternalResource.class.getDeclaredFields();
+    assertThat(fields.length).as("Fields: " + Arrays.asList(fields)).isEqualTo(0);
+  }
+
+  @Test
+  public void isSerializable() throws Exception {
+    assertThat(SerializableExternalResource.class).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void canBeSerialized() throws Throwable {
+    FakeSerializableExternalResource instance = new FakeSerializableExternalResource().value(1);
+
+    FakeSerializableExternalResource cloned = (FakeSerializableExternalResource) SerializationUtils.clone(instance);
+
+    assertThat(instance.value()).isEqualTo(1);
+    assertThat(cloned.value()).isEqualTo(1);
+
+    instance.value(2);
+
+    assertThat(instance.value()).isEqualTo(2);
+    assertThat(cloned.value()).isEqualTo(1);
+  }
+
+  /**
+   * Fake SerializableExternalResource with a simple int field.
+   */
+  private static class FakeSerializableExternalResource extends SerializableExternalResource {
+
+    private int value = -1;
+
+    public FakeSerializableExternalResource value(final int value) {
+      this.value = value;
+      return this;
+    }
+
+    public int value() {
+      return this.value;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleListTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleListTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleListTest.java
new file mode 100755
index 0000000..ebd4b91
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleListTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import java.io.Serializable;
+
+/**
+ * Unit tests for {@link SerializableRuleList}.
+ */
+@Category(UnitTest.class)
+public class SerializableRuleListTest {
+
+  @Test
+  public void isSerializable() throws Exception {
+    assertThat(SerializableRuleList.class).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void canBeSerialized() throws Exception {
+    String value = "foo";
+    FakeSerializableTestRule fakeRule = new FakeSerializableTestRule().value(value);
+    SerializableRuleList instance = new SerializableRuleList().add(fakeRule);
+
+    SerializableRuleList cloned = (SerializableRuleList) SerializationUtils.clone(instance);
+
+    assertThat(cloned.rules().size()).isEqualTo(1);
+    assertThat(cloned.rules().get(0)).isInstanceOf(FakeSerializableTestRule.class).isEqualTo(fakeRule);
+  }
+
+  /**
+   * Fake SerializableTestRule with a string field and overriding equals.
+   */
+  private static class FakeSerializableTestRule implements SerializableTestRule {
+
+    private String value = null;
+
+    public FakeSerializableTestRule value(final String value) {
+      this.value = value;
+      return this;
+    }
+
+    public String value() {
+      return this.value;
+    }
+
+    @Override
+    public Statement apply(final Statement base, final Description description) {
+      return new Statement() {
+        @Override
+        public void evaluate() throws Throwable {
+          base.evaluate();
+        }
+      };
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      FakeSerializableTestRule that = (FakeSerializableTestRule) o;
+
+      return this.value != null ? this.value.equals(that.value()) : that.value() == null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolderTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolderTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolderTest.java
new file mode 100755
index 0000000..b52787e
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolderTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldSerializationUtils.*;
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldsOfTemporaryFolder.*;
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/**
+ * Unit tests for {@link SerializableTemporaryFolder}.
+ */
+@Category(UnitTest.class)
+public class SerializableTemporaryFolderTest {
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Test
+  public void hasTwoFields() throws Exception {
+    Field[] fields = TemporaryFolder.class.getDeclaredFields();
+    assertThat(fields.length).as("Fields: " + Arrays.asList(fields)).isEqualTo(2);
+  }
+
+  @Test
+  public void fieldParentFolderShouldExist() throws Exception {
+    Field field = TemporaryFolder.class.getDeclaredField(FIELD_PARENT_FOLDER);
+    assertThat(field.getType()).isEqualTo(File.class);
+  }
+
+  @Test
+  public void fieldFolderShouldExist() throws Exception {
+    Field field = TemporaryFolder.class.getDeclaredField(FIELD_FOLDER);
+    assertThat(field.getType()).isEqualTo(File.class);
+  }
+
+  @Test
+  public void fieldsCanBeRead() throws Exception {
+    File parentFolder = this.temporaryFolder.getRoot();
+
+    SerializableTemporaryFolder instance = new SerializableTemporaryFolder(parentFolder);
+    instance.create();
+
+    assertThat(readField(TemporaryFolder.class, instance, FIELD_PARENT_FOLDER)).isEqualTo(parentFolder);
+    assertThat(readField(TemporaryFolder.class, instance, FIELD_FOLDER)).isEqualTo(instance.getRoot());
+  }
+
+  @Test
+  public void isSerializable() throws Exception {
+    assertThat(SerializableTemporaryFolder.class).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void canBeSerialized() throws Exception {
+    File parentFolder = this.temporaryFolder.getRoot();
+
+    SerializableTemporaryFolder instance = new SerializableTemporaryFolder(parentFolder);
+    instance.create();
+
+    SerializableTemporaryFolder cloned = (SerializableTemporaryFolder)SerializationUtils.clone(instance);
+
+    assertThat(readField(TemporaryFolder.class, cloned, FIELD_PARENT_FOLDER)).isEqualTo(parentFolder);
+    assertThat(readField(TemporaryFolder.class, cloned, FIELD_FOLDER)).isEqualTo(cloned.getRoot());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestNameTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestNameTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestNameTest.java
new file mode 100755
index 0000000..9485816
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestNameTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldsOfTestName.*;
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.runner.Description;
+
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/**
+ * Unit tests for {@link SerializableTestName}.
+ */
+@Category(UnitTest.class)
+public class SerializableTestNameTest {
+
+  @Test
+  public void hasOneFields() throws Exception {
+    Field[] fields = TestName.class.getDeclaredFields();
+    assertThat(fields.length).as("Fields: " + Arrays.asList(fields)).isEqualTo(1);
+  }
+
+  @Test
+  public void fieldNameShouldExist() throws Exception {
+    Field field = TestName.class.getDeclaredField(FIELD_NAME);
+    assertThat(field.getType()).isEqualTo(String.class);
+  }
+
+  @Test
+  public void fieldsCanBeRead() throws Exception {
+    String name = "foo";
+    Description mockDescription = mock(Description.class);
+    when(mockDescription.getMethodName()).thenReturn(name);
+
+    SerializableTestName instance = new SerializableTestName();
+    instance.starting(mockDescription);
+
+    assertThat(instance.getMethodName()).isEqualTo(name);
+  }
+
+  @Test
+  public void isSerializable() throws Exception {
+    assertThat(SerializableTestName.class).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void canBeSerialized() throws Exception {
+    String name = "bar";
+    Description mockDescription = mock(Description.class);
+    when(mockDescription.getMethodName()).thenReturn(name);
+
+    SerializableTestName instance = new SerializableTestName();
+    instance.starting(mockDescription);
+
+    assertThat(instance.getMethodName()).isEqualTo(name);
+
+    SerializableTestName cloned = (SerializableTestName) SerializationUtils.clone(instance);
+
+    assertThat(cloned.getMethodName()).isEqualTo(name);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcherTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcherTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcherTest.java
new file mode 100755
index 0000000..1e8c1c4
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcherTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestWatcher;
+
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+
+/**
+ * Unit tests for {@link SerializableTestWatcher}.
+ */
+@Category(UnitTest.class)
+public class SerializableTestWatcherTest {
+
+  @Test
+  public void hasZeroFields() throws Exception {
+    Field[] fields = TestWatcher.class.getDeclaredFields();
+    assertThat(fields.length).as("Fields: " + Arrays.asList(fields)).isEqualTo(0);
+  }
+
+  @Test
+  public void isSerializable() throws Exception {
+    assertThat(SerializableTestWatcher.class).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void canBeSerialized() throws Exception {
+    FakeSerializableTestWatcher instance = new FakeSerializableTestWatcher().value(1);
+
+    FakeSerializableTestWatcher cloned = (FakeSerializableTestWatcher) SerializationUtils.clone(instance);
+
+    assertThat(instance.value()).isEqualTo(1);
+    assertThat(cloned.value()).isEqualTo(1);
+
+    instance.value(2);
+
+    assertThat(instance.value()).isEqualTo(2);
+    assertThat(cloned.value()).isEqualTo(1);
+  }
+
+  /**
+   * Fake SerializableTestWatcher with a simple int field.
+   */
+  private static class FakeSerializableTestWatcher extends SerializableTestWatcher {
+
+    private int value = -1;
+
+    private FakeSerializableTestWatcher value(final int value) {
+      this.value = value;
+      return this;
+    }
+
+    private int value() {
+      return this.value;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeoutTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeoutTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeoutTest.java
new file mode 100755
index 0000000..c266c63
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeoutTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldSerializationUtils.*;
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldsOfTimeout.*;
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.commons.lang.SerializationUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.Timeout;
+
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Unit tests for {@link SerializableTimeout}.
+ */
+@Category(UnitTest.class)
+public class SerializableTimeoutTest {
+
+  @Test
+  public void hasThreeFields() throws Exception {
+    Field[] fields = Timeout.class.getDeclaredFields();
+    assertThat(fields.length).as("Fields: " + Arrays.asList(fields)).isEqualTo(3);
+  }
+
+  @Test
+  public void fieldTimeoutShouldExist() throws Exception {
+    Field field = Timeout.class.getDeclaredField(FIELD_TIMEOUT);
+    assertThat(field.getType()).isEqualTo(Long.TYPE);
+  }
+
+  @Test
+  public void fieldTimeUnitShouldExist() throws Exception {
+    Field field = Timeout.class.getDeclaredField(FIELD_TIME_UNIT);
+    assertThat(field.getType()).isEqualTo(TimeUnit.class);
+  }
+
+  @Test
+  public void fieldLookForStuckThreadShouldExist() throws Exception {
+    Field field = Timeout.class.getDeclaredField(FIELD_LOOK_FOR_STUCK_THREAD);
+    assertThat(field.getType()).isEqualTo(Boolean.TYPE);
+  }
+
+  @Test
+  public void fieldsCanBeRead() throws Exception {
+    long timeout = 1000;
+    TimeUnit timeUnit = TimeUnit.MILLISECONDS;
+    boolean lookingForStuckThread = false;
+
+    SerializableTimeout instance = SerializableTimeout.builder()
+        .withTimeout(timeout, timeUnit)
+        .withLookingForStuckThread(lookingForStuckThread)
+        .build();
+
+    assertThat(readField(Timeout.class, instance, FIELD_TIMEOUT)).isEqualTo(timeout);
+    assertThat(readField(Timeout.class, instance, FIELD_TIME_UNIT)).isEqualTo(timeUnit);
+    assertThat(readField(Timeout.class, instance, FIELD_LOOK_FOR_STUCK_THREAD)).isEqualTo(lookingForStuckThread);
+  }
+
+  @Test
+  public void isSerializable() throws Exception {
+    assertThat(SerializableTimeout.class).isInstanceOf(Serializable.class);
+  }
+
+  @Test
+  public void canBeSerialized() throws Exception {
+    long timeout = 2;
+    TimeUnit timeUnit = TimeUnit.SECONDS;
+    boolean lookingForStuckThread = true;
+
+    SerializableTimeout instance = SerializableTimeout.builder()
+        .withTimeout(timeout, timeUnit)
+        .withLookingForStuckThread(lookingForStuckThread)
+        .build();
+
+    assertThat(readField(Timeout.class, instance, FIELD_TIMEOUT)).isEqualTo(timeout);
+    assertThat(readField(Timeout.class, instance, FIELD_TIME_UNIT)).isEqualTo(timeUnit);
+    assertThat(readField(Timeout.class, instance, FIELD_LOOK_FOR_STUCK_THREAD)).isEqualTo(lookingForStuckThread);
+
+    SerializableTimeout cloned = (SerializableTimeout) SerializationUtils.clone(instance);
+
+    assertThat(readField(Timeout.class, cloned, FIELD_TIMEOUT)).isEqualTo(timeout);
+    assertThat(readField(Timeout.class, cloned, FIELD_TIME_UNIT)).isEqualTo(timeUnit);
+    assertThat(readField(Timeout.class, cloned, FIELD_LOOK_FOR_STUCK_THREAD)).isEqualTo(lookingForStuckThread);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleTest.java
deleted file mode 100755
index b67d1eb..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/ExpectedTimeoutRuleTest.java
+++ /dev/null
@@ -1,214 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.ExpectedTimeoutRule;
-
-/**
- * Unit tests for ExpectedTimeout JUnit Rule.
- * 
- * @author Kirk Lund
- * @since 8.2
- */
-@Category(UnitTest.class)
-public class ExpectedTimeoutRuleTest {
-
-  @Test
-  public void passesUnused() {
-    Result result = runTest(PassingTestShouldPassWhenUnused.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-  }
-  
-  @Test
-  public void failsWithoutExpectedException() {
-    Result result = runTest(FailsWithoutExpectedException.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-    
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw an instance of " + TimeoutException.class.getName());
-  }
-  
-  @Test
-  public void failsWithoutExpectedTimeoutException() {
-    Result result = runTest(FailsWithoutExpectedTimeoutException.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-    
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWithoutExpectedTimeoutException.message + "\")");
-  }
-  
-  @Test
-  public void failsWithExpectedTimeoutButWrongError() {
-    Result result = runTest(FailsWithExpectedTimeoutButWrongError.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-    
-    Failure failure = failures.get(0);
-    String expectedMessage = 
-        "\n" + 
-        "Expected: (an instance of java.util.concurrent.TimeoutException and exception with message a string containing \"this is a message for FailsWithExpectedTimeoutButWrongError\")" +
-        "\n" + 
-        "     " +
-        "but: an instance of java.util.concurrent.TimeoutException <java.lang.NullPointerException> is a java.lang.NullPointerException";
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessageContaining(expectedMessage);
-  }
-  
-  @Test
-  public void passesWithExpectedTimeoutAndTimeoutException() {
-    Result result = runTest(PassesWithExpectedTimeoutAndTimeoutException.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-  }
-  
-  @Test
-  public void failsWhenTimeoutIsEarly() {
-    Result result = runTest(FailsWhenTimeoutIsEarly.class);
-   
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-    
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsEarly.message + "\")");
-  }
-  
-  @Test
-  public void failsWhenTimeoutIsLate() {
-    Result result = runTest(FailsWhenTimeoutIsLate.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-    
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsLate.message + "\")");
-  }
-  
-  public static class AbstractExpectedTimeoutRuleTest {
-    @Rule
-    public ExpectedTimeoutRule timeout = ExpectedTimeoutRule.none();
-  }
-  
-  public static class PassingTestShouldPassWhenUnused extends AbstractExpectedTimeoutRuleTest {
-    @Test
-    public void passesUnused() throws Exception {
-    }
-  }
-  
-  public static class FailsWithoutExpectedException extends AbstractExpectedTimeoutRuleTest {
-    @Test
-    public void failsWithoutExpectedException() throws Exception {
-      timeout.expect(TimeoutException.class);
-    }
-  }
-  
-  public static class FailsWithoutExpectedTimeoutException extends AbstractExpectedTimeoutRuleTest {
-    public static final String message = "this is a message for FailsWithoutExpectedTimeoutException";
-    @Test
-    public void failsWithoutExpectedTimeoutAndTimeoutException() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(1000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-    }
-  }
-  
-  public static class FailsWithExpectedTimeoutButWrongError extends AbstractExpectedTimeoutRuleTest {
-    public static final String message = "this is a message for FailsWithExpectedTimeoutButWrongError";
-    @Test
-    public void failsWithExpectedTimeoutButWrongError() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(1000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-      throw new NullPointerException();
-    }
-  }
-
-  public static class PassesWithExpectedTimeoutAndTimeoutException extends AbstractExpectedTimeoutRuleTest {
-    public static final String message = "this is a message for PassesWithExpectedTimeoutAndTimeoutException";
-    public static final Class<TimeoutException> exceptionClass = TimeoutException.class;
-    @Test
-    public void passesWithExpectedTimeoutAndTimeoutException() throws Exception {
-      timeout.expect(exceptionClass);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(1000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-      throw new TimeoutException(message);
-    }
-  }
-
-  public static class FailsWhenTimeoutIsEarly extends AbstractExpectedTimeoutRuleTest {
-    public static final String message = "this is a message for FailsWhenTimeoutIsEarly";
-    @Test
-    public void failsWhenTimeoutIsEarly() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(1000);
-      timeout.expectMaximumDuration(2000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(10);
-    }
-  }
-
-  public static class FailsWhenTimeoutIsLate extends AbstractExpectedTimeoutRuleTest {
-    public static final String message = "this is a message for FailsWhenTimeoutIsLate";
-    @Test
-    public void failsWhenTimeoutIsLate() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(20);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/IgnoreUntilRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/IgnoreUntilRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/IgnoreUntilRuleTest.java
deleted file mode 100755
index a984d1d..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/IgnoreUntilRuleTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.List;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.IgnoreUntil;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.IgnoreUntilRule;
-
-/**
- * Unit tests for IgnoreUntil JUnit Rule
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class IgnoreUntilRuleTest {
-
-  private static final String ASSERTION_ERROR_MESSAGE = "failing test";
-  
-  @Test
-  public void shouldIgnoreWhenUntilIsInFuture() {
-    Result result = runTest(ShouldIgnoreWhenUntilIsInFuture.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(ShouldIgnoreWhenUntilIsInFuture.count).isEqualTo(0);
-  }
-  
-  @Test
-  public void shouldExecuteWhenUntilIsInPast() {
-    Result result = runTest(ShouldExecuteWhenUntilIsInPast.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
-    assertThat(ShouldExecuteWhenUntilIsInPast.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void shouldExecuteWhenUntilIsDefault() {
-    Result result = runTest(ShouldExecuteWhenUntilIsDefault.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
-    assertThat(ShouldExecuteWhenUntilIsDefault.count).isEqualTo(1);
-  }
-  
-  public static class ShouldIgnoreWhenUntilIsInFuture {
-    private static int count;
-    
-    @Rule
-    public final IgnoreUntilRule ignoreUntilRule = new IgnoreUntilRule();
-    
-    @Test
-    @IgnoreUntil(value = "description", until = "3000-01-01")
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-
-  public static class ShouldExecuteWhenUntilIsInPast {
-    private static int count;
-    
-    @Rule
-    public final IgnoreUntilRule ignoreUntilRule = new IgnoreUntilRule();
-    
-    @Test
-    @IgnoreUntil(value = "description", until = "1980-01-01")
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-
-  public static class ShouldExecuteWhenUntilIsDefault {
-    private static int count;
-    
-    @Rule
-    public final IgnoreUntilRule ignoreUntilRule = new IgnoreUntilRule();
-    
-    @Test
-    @IgnoreUntil(value = "description")
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/JUnitRuleTestSuite.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/JUnitRuleTestSuite.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/JUnitRuleTestSuite.java
deleted file mode 100755
index 4c9e315..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/JUnitRuleTestSuite.java
+++ /dev/null
@@ -1,33 +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.junit.rules.tests;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-  ExpectedTimeoutRuleTest.class,
-  IgnoreUntilRuleTest.class,
-  RepeatRuleTest.class,
-  RetryRuleGlobalWithErrorTest.class,
-  RetryRuleGlobalWithExceptionTest.class,
-  RetryRuleLocalWithErrorTest.class,
-  RetryRuleLocalWithExceptionTest.class,
-})
-public class JUnitRuleTestSuite {
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RepeatRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RepeatRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RepeatRuleTest.java
deleted file mode 100755
index 2d37de4..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RepeatRuleTest.java
+++ /dev/null
@@ -1,304 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.List;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.Repeat;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.RepeatRule;
-
-/**
- * Unit tests for Repeat JUnit Rule.
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class RepeatRuleTest {
-
-  private static final String ASSERTION_ERROR_MESSAGE = "failing test";
-  
-  @Test
-  public void failingTestShouldFailOneTimeWhenRepeatIsUnused() {
-    Result result = runTest(FailingTestShouldFailOneTimeWhenRepeatIsUnused.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
-    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsUnused.count).isEqualTo(1);
-  }
-
-  @Test
-  public void passingTestShouldPassOneTimeWhenRepeatIsUnused() {
-    Result result = runTest(PassingTestShouldPassOneTimeWhenRepeatIsUnused.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassingTestShouldPassOneTimeWhenRepeatIsUnused.count).isEqualTo(1);
-  }
-
-  @Test
-  public void zeroValueShouldThrowIllegalArgumentException() {
-    Result result = runTest(ZeroValueShouldThrowIllegalArgumentException.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage("Repeat value must be a positive integer");
-    assertThat(ZeroValueShouldThrowIllegalArgumentException.count).isEqualTo(0);
-  }
-  
-  @Test
-  public void negativeValueShouldThrowIllegalArgumentException() {
-    Result result = runTest(NegativeValueShouldThrowIllegalArgumentException.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage("Repeat value must be a positive integer");
-    assertThat(NegativeValueShouldThrowIllegalArgumentException.count).isEqualTo(0);
-  }
-
-  @Test
-  public void failingTestShouldFailOneTimeWhenRepeatIsOne() {
-    Result result = runTest(FailingTestShouldFailOneTimeWhenRepeatIsOne.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
-    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsOne.count).isEqualTo(1);
-  }
-
-  @Test
-  public void passingTestShouldPassOneTimeWhenRepeatIsOne() {
-    Result result = runTest(PassingTestShouldPassOneTimeWhenRepeatIsOne.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassingTestShouldPassOneTimeWhenRepeatIsOne.count).isEqualTo(1);
-  }
-
-  @Test
-  public void failingTestShouldFailOneTimeWhenRepeatIsTwo() {
-    Result result = runTest(FailingTestShouldFailOneTimeWhenRepeatIsTwo.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
-    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsTwo.count).isEqualTo(1);
-  }
-
-  @Test
-  public void passingTestShouldPassTwoTimesWhenRepeatIsTwo() {
-    Result result = runTest(PassingTestShouldPassTwoTimesWhenRepeatIsTwo.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassingTestShouldPassTwoTimesWhenRepeatIsTwo.count).isEqualTo(2);
-  }
-
-  @Test
-  public void failingTestShouldFailOneTimeWhenRepeatIsThree() {
-    Result result = runTest(FailingTestShouldFailOneTimeWhenRepeatIsThree.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
-    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsThree.count).isEqualTo(1);
-  }
-
-  @Test
-  public void passingTestShouldPassThreeTimesWhenRepeatIsThree() {
-    Result result = runTest(PassingTestShouldPassThreeTimesWhenRepeatIsThree.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassingTestShouldPassThreeTimesWhenRepeatIsThree.count).isEqualTo(3);
-  }
-
-  public static class FailingTestShouldFailOneTimeWhenRepeatIsUnused {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-
-  public static class PassingTestShouldPassOneTimeWhenRepeatIsUnused {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-
-  public static class ZeroValueShouldThrowIllegalArgumentException {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(0)
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-
-  public static class NegativeValueShouldThrowIllegalArgumentException {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(-1)
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-
-  public static class PassingTestShouldBeSkippedWhenRepeatIsZero {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(0)
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailingTestShouldFailOneTimeWhenRepeatIsOne {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(1)
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-
-  public static class PassingTestShouldPassOneTimeWhenRepeatIsOne {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(1)
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-
-  public static class FailingTestShouldFailOneTimeWhenRepeatIsTwo {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(2)
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-
-  public static class PassingTestShouldPassTwoTimesWhenRepeatIsTwo {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(2)
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-
-  public static class FailingTestShouldFailOneTimeWhenRepeatIsThree {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(3)
-    public void doTest() throws Exception {
-      count++;
-      fail(ASSERTION_ERROR_MESSAGE);
-    }
-  }
-
-  public static class PassingTestShouldPassThreeTimesWhenRepeatIsThree {
-    protected static int count = 0;
-    
-    @Rule
-    public RepeatRule repeat = new RepeatRule();
-
-    @Test
-    @Repeat(3)
-    public void doTest() throws Exception {
-      count++;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithErrorTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithErrorTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithErrorTest.java
deleted file mode 100755
index 78bc410..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithErrorTest.java
+++ /dev/null
@@ -1,250 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-import static org.junit.Assert.fail;
-
-import java.util.List;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.Retry;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.RetryRule;
-
-/**
- * Unit tests for Retry JUnit Rule involving global scope (ie Rule affects all 
- * tests in the test case) with failures due to an Exception.
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class RetryRuleGlobalWithErrorTest {
-  
-  @Test
-  public void zeroIsIllegal() {
-    Result result = runTest(ZeroIsIllegal.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage(ZeroIsIllegal.message);
-    assertThat(ZeroIsIllegal.count).isEqualTo(0);
-  }
-  
-  @Test
-  public void failsWithOne() {
-    Result result = runTest(FailsWithOne.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsWithOne.message);
-    assertThat(FailsWithOne.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void passesWithOne() {
-    Result result = runTest(PassesWithOne.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesWithOne.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void passesWithUnused() {
-    Result result = runTest(PassesWhenUnused.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesWhenUnused.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void failsOnSecondAttempt() {
-    Result result = runTest(FailsOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnSecondAttempt.message);
-    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
-  }
-
-  @Test
-  public void passesOnSecondAttempt() {
-    Result result = runTest(PassesOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
-  }
-  
-  @Test
-  public void failsOnThirdAttempt() {
-    Result result = runTest(FailsOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnThirdAttempt.message);
-    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
-  }
-
-  @Test
-  public void passesOnThirdAttempt() {
-    Result result = runTest(PassesOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
-  }
-  
-  public static class ZeroIsIllegal {
-    protected static int count;
-    protected static final String message = "Retry count must be greater than zero";
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(0);
-
-    @Test
-    public void zeroIsIllegal() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailsWithOne {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(1);
-
-    @Test
-    public void failsWithOne() throws Exception {
-      count++;
-      message = "Failing " + count;
-      fail(message);
-    }
-  }
-  
-  public static class PassesWithOne {
-    protected static int count;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(1);
-
-    @Test
-    public void passesWithOne() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class PassesWhenUnused {
-    protected static int count;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(2);
-
-    @Test
-    public void passesWithUnused() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailsOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(2);
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() {
-      count++;
-      message = "Failing " + count;
-      fail(message);
-    }
-  }
-  
-  public static class PassesOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule(2);
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() {
-      count++;
-      if (count < 2) {
-        message = "Failing " + count;
-        fail(message);
-      }
-    }
-  }
-  
-  public static class FailsOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(3);
-
-    @Test
-    @Retry(3)
-    public void failsOnThirdAttempt() {
-      count++;
-      message = "Failing " + count;
-      fail(message);
-    }
-  }
-
-  public static class PassesOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(3);
-
-    @Test
-    public void failsOnThirdAttempt() {
-      count++;
-      if (count < 3) {
-        message = "Failing " + count;
-        fail(message);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithExceptionTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithExceptionTest.java
deleted file mode 100755
index 114eeb4..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleGlobalWithExceptionTest.java
+++ /dev/null
@@ -1,254 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.List;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.Retry;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.RetryRule;
-
-/**
- * Unit tests for Retry JUnit Rule involving global scope (ie Rule affects all 
- * tests in the test case) with failures due to an Exception.
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class RetryRuleGlobalWithExceptionTest {
-  
-  @Test
-  public void zeroIsIllegal() {
-    Result result = runTest(ZeroIsIllegal.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage(ZeroIsIllegal.message);
-    assertThat(ZeroIsIllegal.count).isEqualTo(0);
-  }
-  
-  @Test
-  public void failsWithOne() {
-    Result result = runTest(FailsWithOne.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsWithOne.message);
-    assertThat(FailsWithOne.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void passesWithOne() {
-    Result result = runTest(PassesWithOne.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-  }
-  
-  @Test
-  public void passesWithUnused() {
-    Result result = runTest(PassesWhenUnused.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-  }
-  
-  @Test
-  public void failsOnSecondAttempt() {
-    Result result = runTest(FailsOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnSecondAttempt.message);
-    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
-  }
-
-  @Test
-  public void passesOnSecondAttempt() {
-    Result result = runTest(PassesOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
-  }
-  
-  @Test
-  public void failsOnThirdAttempt() {
-    Result result = runTest(FailsOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnThirdAttempt.message);
-    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
-  }
-
-  @Test
-  public void passesOnThirdAttempt() {
-    Result result = runTest(PassesOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
-  }
-  
-  public static class CustomException extends Exception {
-    private static final long serialVersionUID = 1L;
-    public CustomException(final String message) {
-      super(message);
-    }
-  }
-  
-  public static class ZeroIsIllegal {
-    protected static int count;
-    protected static final String message = "Retry count must be greater than zero";
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(0);
-
-    @Test
-    public void zeroIsIllegal() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailsWithOne {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(1);
-
-    @Test
-    public void failsWithOne() throws Exception {
-      count++;
-      message = "Failing " + count;
-      throw new CustomException(message);
-    }
-  }
-  
-  public static class PassesWithOne {
-    protected static int count;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(1);
-
-    @Test
-    public void passesWithOne() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class PassesWhenUnused {
-    protected static int count;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(2);
-
-    @Test
-    public void passesWithUnused() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailsOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(2);
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() throws Exception {
-      count++;
-      message = "Failing " + count;
-      throw new CustomException(message);
-    }
-  }
-  
-  public static class PassesOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule(2);
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() throws Exception {
-      count++;
-      if (count < 2) {
-        message = "Failing " + count;
-        throw new CustomException(message);
-      }
-    }
-  }
-  
-  public static class FailsOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(3);
-
-    @Test
-    @Retry(3)
-    public void failsOnThirdAttempt() throws Exception {
-      count++;
-      message = "Failing " + count;
-      throw new CustomException(message);
-    }
-  }
-
-  public static class PassesOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule(3);
-
-    @Test
-    public void failsOnThirdAttempt() throws Exception {
-      count++;
-      if (count < 3) {
-        message = "Failing " + count;
-        throw new CustomException(message);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithErrorTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithErrorTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithErrorTest.java
deleted file mode 100755
index af5a853..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithErrorTest.java
+++ /dev/null
@@ -1,207 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-import static org.junit.Assert.fail;
-
-import java.util.List;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.Retry;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.RetryRule;
-
-/**
- * Unit tests for Retry JUnit Rule involving local scope (ie Rule affects 
- * test methods annotated with @Retry) with failures due to an Error.
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class RetryRuleLocalWithErrorTest {
-
-  @Test
-  public void failsUnused() {
-    Result result = runTest(FailsUnused.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsUnused.message);
-    assertThat(FailsUnused.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void passesUnused() {
-    Result result = runTest(PassesUnused.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesUnused.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void failsOnSecondAttempt() {
-    Result result = runTest(FailsOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnSecondAttempt.message);
-    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
-  }
-
-  @Test
-  public void passesOnSecondAttempt() {
-    Result result = runTest(PassesOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
-  }
-  
-  @Test
-  public void failsOnThirdAttempt() {
-    Result result = runTest(FailsOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnThirdAttempt.message);
-    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
-  }
-
-  @Test
-  public void passesOnThirdAttempt() {
-    Result result = runTest(PassesOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
-  }
-  
-  public static class FailsUnused {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    public void failsUnused() throws Exception {
-      count++;
-      message = "Failing " + count;
-      fail(message);
-    }
-  }
-  
-  public static class PassesUnused {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    public void passesUnused() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailsOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() {
-      count++;
-      message = "Failing " + count;
-      fail(message);
-    }
-  }
-  
-  public static class PassesOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() {
-      count++;
-      if (count < 2) {
-        message = "Failing " + count;
-        fail(message);
-      }
-    }
-  }
-  
-  public static class FailsOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(3)
-    public void failsOnThirdAttempt() {
-      count++;
-
-      message = "Failing " + count;
-      fail(message);
-    }
-  }
-
-  public static class PassesOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(3)
-    public void failsOnThirdAttempt() {
-      count++;
-
-      if (count < 3) {
-        message = "Failing " + count;
-        fail(message);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithExceptionTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithExceptionTest.java
deleted file mode 100755
index 1b16a7a..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RetryRuleLocalWithExceptionTest.java
+++ /dev/null
@@ -1,213 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import java.util.List;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.Retry;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.RetryRule;
-
-/**
- * Unit tests for Retry JUnit Rule involving local scope (ie Rule affects 
- * test methods annotated with @Retry) with failures due to an Exception.
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class RetryRuleLocalWithExceptionTest {
-
-  @Test
-  public void failsUnused() {
-    Result result = runTest(FailsUnused.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsUnused.message);
-    assertThat(FailsUnused.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void passesUnused() {
-    Result result = runTest(PassesUnused.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesUnused.count).isEqualTo(1);
-  }
-  
-  @Test
-  public void failsOnSecondAttempt() {
-    Result result = runTest(FailsOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnSecondAttempt.message);
-    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
-  }
-
-  @Test
-  public void passesOnSecondAttempt() {
-    Result result = runTest(PassesOnSecondAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
-  }
-  
-  @Test
-  public void failsOnThirdAttempt() {
-    Result result = runTest(FailsOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isFalse();
-    
-    List<Failure> failures = result.getFailures();
-    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
-
-    Failure failure = failures.get(0);
-    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnThirdAttempt.message);
-    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
-  }
-
-  @Test
-  public void passesOnThirdAttempt() {
-    Result result = runTest(PassesOnThirdAttempt.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
-  }
-  
-  public static class CustomException extends Exception {
-    private static final long serialVersionUID = 1L;
-    public CustomException(final String message) {
-      super(message);
-    }
-  }
-  
-  public static class FailsUnused {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    public void failsUnused() throws Exception {
-      count++;
-      message = "Failing " + count;
-      throw new CustomException(message);
-    }
-  }
-  
-  public static class PassesUnused {
-    protected static int count;
-    protected static String message;
-
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    public void passesUnused() throws Exception {
-      count++;
-    }
-  }
-  
-  public static class FailsOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() throws Exception {
-      count++;
-      message = "Failing " + count;
-      throw new CustomException(message);
-    }
-  }
-  
-  public static class PassesOnSecondAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(2)
-    public void failsOnSecondAttempt() throws Exception {
-      count++;
-      if (count < 2) {
-        message = "Failing " + count;
-        throw new CustomException(message);
-      }
-    }
-  }
-  
-  public static class FailsOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(3)
-    public void failsOnThirdAttempt() throws Exception {
-      count++;
-
-      message = "Failing " + count;
-      throw new CustomException(message);
-    }
-  }
-
-  public static class PassesOnThirdAttempt {
-    protected static int count;
-    protected static String message;
-    
-    @Rule
-    public RetryRule retryRule = new RetryRule();
-
-    @Test
-    @Retry(3)
-    public void failsOnThirdAttempt() throws Exception {
-      count++;
-
-      if (count < 3) {
-        message = "Failing " + count;
-        throw new CustomException(message);
-      }
-    }
-  }
-}



[6/7] incubator-geode git commit: GEODE-953: Fix missing junit category

Posted by kl...@apache.org.
GEODE-953: Fix missing junit category


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

Branch: refs/heads/feature/GEODE-949-2
Commit: b4ed2aa5087fab377ce5d2bd55350fd5b42a54a4
Parents: 5342935
Author: Kirk Lund <kl...@pdx2-office-dhcp32.eng.vmware.com>
Authored: Mon Feb 29 14:52:21 2016 -0800
Committer: Kirk Lund <kl...@pdx2-office-dhcp32.eng.vmware.com>
Committed: Mon Feb 29 14:52:21 2016 -0800

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b4ed2aa5/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
index ce98dfe..913e27d 100755
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
@@ -18,10 +18,12 @@ package com.gemstone.gemfire.test.junit.rules;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.Result;
 
@@ -30,6 +32,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 /**
  * Unit tests for {@link RuleList}.
  */
+@Category(UnitTest.class)
 public class RuleListTest {
 
   private static AtomicInteger counter;


[5/7] incubator-geode git commit: GEODE-953: Cleanup geode-junit

Posted by kl...@apache.org.
GEODE-953: Cleanup geode-junit

* cleanup geode-junit usage of JUnit Rules to ensure JUnit code is always extended rather than copied
* change CRLF to LF
* remove @author tags
* fix all javadocs
* fix formatting
* add new tests
* remove ExpectedTimeout and ExpectedTimeoutTest (they were dupes of ExpectedTimeoutRule and ExpectedTimeoutRuleTest)
* use serialization by reflection for serializable subclasses of JUnit rules that lack sufficient ability to subclass
* repackage rule tests to same package as rules
* repackage serializable rules into serializable sub-package


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

Branch: refs/heads/feature/GEODE-949-2
Commit: 5342935d301a4026b289637885e9236414feeb78
Parents: 1e31fa6
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Feb 29 13:06:11 2016 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Mon Feb 29 13:06:11 2016 -0800

----------------------------------------------------------------------
 .../internal/process/PidFileJUnitTest.java      |   4 +-
 .../dunit/rules/DistributedDisconnectRule.java  |  44 +-
 .../rules/DistributedExternalResource.java      |  27 +-
 .../DistributedRestoreSystemProperties.java     |   5 +-
 .../gemfire/test/dunit/rules/RemoteInvoker.java |  10 +-
 geode-junit/build.gradle                        |   1 +
 .../gemfire/test/junit/ConditionalIgnore.java   |   1 -
 .../gemfire/test/junit/IgnoreCondition.java     |   1 -
 .../gemfire/test/junit/IgnoreUntil.java         |   1 -
 .../com/gemstone/gemfire/test/junit/Repeat.java |   3 +-
 .../com/gemstone/gemfire/test/junit/Retry.java  |   4 +-
 .../test/junit/categories/ContainerTest.java    |   3 +-
 .../test/junit/categories/DistributedTest.java  |   3 +-
 .../categories/DistributedTransactionsTest.java |   3 +-
 .../test/junit/categories/HydraTest.java        |   3 +-
 .../test/junit/categories/IntegrationTest.java  |   3 +-
 .../test/junit/categories/PerformanceTest.java  |   3 +-
 .../gemfire/test/junit/categories/UITest.java   |   3 +-
 .../gemfire/test/junit/categories/UnitTest.java |  13 +-
 .../gemfire/test/junit/categories/WanTest.java  |   5 +-
 .../test/junit/rules/ConditionalIgnoreRule.java |   1 -
 .../test/junit/rules/ExpectedTimeout.java       | 180 --------
 .../test/junit/rules/ExpectedTimeoutRule.java   |  42 +-
 .../test/junit/rules/IgnoreUntilRule.java       |   1 -
 .../gemfire/test/junit/rules/RepeatRule.java    |   3 +-
 .../gemfire/test/junit/rules/RetryRule.java     |   1 -
 .../gemfire/test/junit/rules/RuleList.java      |  95 +++++
 .../rules/SerializableExternalResource.java     | 107 -----
 .../test/junit/rules/SerializableRuleChain.java | 119 ------
 .../rules/SerializableTemporaryFolder.java      |  70 ----
 .../test/junit/rules/SerializableTestName.java  |  54 ---
 .../test/junit/rules/SerializableTestRule.java  |  33 --
 .../junit/rules/SerializableTestWatcher.java    |  29 --
 .../test/junit/rules/SerializableTimeout.java   | 119 ------
 .../serializable/FieldSerializationUtils.java   |  48 +++
 .../serializable/FieldsOfTemporaryFolder.java   |  26 ++
 .../rules/serializable/FieldsOfTestName.java    |  24 ++
 .../rules/serializable/FieldsOfTimeout.java     |  26 ++
 .../SerializableExternalResource.java           |  25 ++
 .../serializable/SerializableRuleList.java      |  78 ++++
 .../SerializableTemporaryFolder.java            |  70 ++++
 .../serializable/SerializableTestName.java      |  65 +++
 .../serializable/SerializableTestRule.java      |  28 ++
 .../serializable/SerializableTestWatcher.java   |  26 ++
 .../rules/serializable/SerializableTimeout.java | 104 +++++
 .../junit/support/DefaultIgnoreCondition.java   |   3 +-
 .../IgnoreConditionEvaluationException.java     |   1 -
 .../junit/rules/ExpectedTimeoutJUnitTest.java   | 204 ---------
 .../junit/rules/ExpectedTimeoutRuleTest.java    | 246 +++++++++++
 .../test/junit/rules/IgnoreUntilRuleTest.java   | 145 +++++++
 .../test/junit/rules/RepeatRuleTest.java        | 411 +++++++++++++++++++
 .../rules/RetryRuleGlobalWithErrorTest.java     | 326 +++++++++++++++
 .../rules/RetryRuleGlobalWithExceptionTest.java | 332 +++++++++++++++
 .../rules/RetryRuleLocalWithErrorTest.java      | 265 ++++++++++++
 .../rules/RetryRuleLocalWithExceptionTest.java  | 276 +++++++++++++
 .../gemfire/test/junit/rules/RuleListTest.java  | 206 ++++++++++
 .../gemfire/test/junit/rules/TestRunner.java    |  35 ++
 .../examples/RepeatingTestCasesExampleTest.java |  15 +-
 .../rules/examples/RetryRuleExampleTest.java    |  20 +-
 .../rules/examples/RuleAndClassRuleTest.java    | 147 +++++++
 .../SerializableExternalResourceTest.java       |  79 ++++
 .../serializable/SerializableRuleListTest.java  |  89 ++++
 .../SerializableTemporaryFolderTest.java        |  90 ++++
 .../serializable/SerializableTestNameTest.java  |  84 ++++
 .../SerializableTestWatcherTest.java            |  79 ++++
 .../serializable/SerializableTimeoutTest.java   | 106 +++++
 .../rules/tests/ExpectedTimeoutRuleTest.java    | 214 ----------
 .../junit/rules/tests/IgnoreUntilRuleTest.java  | 121 ------
 .../junit/rules/tests/JUnitRuleTestSuite.java   |  33 --
 .../test/junit/rules/tests/RepeatRuleTest.java  | 304 --------------
 .../tests/RetryRuleGlobalWithErrorTest.java     | 250 -----------
 .../tests/RetryRuleGlobalWithExceptionTest.java | 254 ------------
 .../tests/RetryRuleLocalWithErrorTest.java      | 207 ----------
 .../tests/RetryRuleLocalWithExceptionTest.java  | 213 ----------
 .../junit/rules/tests/RuleAndClassRuleTest.java | 138 -------
 .../test/junit/rules/tests/TestRunner.java      |  37 --
 76 files changed, 3620 insertions(+), 2824 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
index 5f81c2b..2906b0e 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/process/PidFileJUnitTest.java
@@ -42,7 +42,7 @@ import org.junit.rules.TemporaryFolder;
 
 import com.gemstone.gemfire.internal.util.StopWatch;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
-import com.gemstone.gemfire.test.junit.rules.ExpectedTimeout;
+import com.gemstone.gemfire.test.junit.rules.ExpectedTimeoutRule;
 
 /**
  * Unit tests the PidFile class.
@@ -60,7 +60,7 @@ public class PidFileJUnitTest {
   public ExpectedException thrown = ExpectedException.none();
   
   @Rule
-  public ExpectedTimeout timeout = ExpectedTimeout.none();
+  public ExpectedTimeoutRule timeout = ExpectedTimeoutRule.none();
   
   protected Mockery mockContext;
   private ExecutorService futures;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedDisconnectRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedDisconnectRule.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedDisconnectRule.java
index 125fc06..47e5175 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedDisconnectRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedDisconnectRule.java
@@ -16,18 +16,18 @@
  */
 package com.gemstone.gemfire.test.dunit.rules;
 
-// TODO: import static com.gemstone.gemfire.test.dunit.DistributedTestRule.*;
+// TODO:uncomment: import static com.gemstone.gemfire.test.dunit.DistributedTestRule.*;
 
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 
-@SuppressWarnings("serial")
+/**
+ * Disconnects all remote DUnit JVMs including the Locator JVM.
+ */
 public class DistributedDisconnectRule extends DistributedExternalResource {
 
   private final boolean disconnectBefore;
   private final boolean disconnectAfter;
-  private final boolean disconnectBeforeClass;
-  private final boolean disconnectAfterClass;
-  
+
   public static Builder builder() {
     return new Builder();
   }
@@ -38,8 +38,6 @@ public class DistributedDisconnectRule extends DistributedExternalResource {
    
   public DistributedDisconnectRule(final RemoteInvoker invoker, final Builder builder) {
     super(invoker);
-    this.disconnectBeforeClass = builder.disconnectBeforeClass;
-    this.disconnectAfterClass = builder.disconnectAfterClass;
     this.disconnectBefore = builder.disconnectBefore;
     this.disconnectAfter = builder.disconnectAfter;
   }
@@ -52,63 +50,35 @@ public class DistributedDisconnectRule extends DistributedExternalResource {
   }
 
   @Override
-  protected void after() throws Throwable {
+  protected void after() {
     if (this.disconnectAfter) {
       invoker().invokeEverywhere(serializableRunnable());
     }
   }
 
-  @Override
-  protected void beforeClass() throws Throwable {
-    if (this.disconnectBeforeClass) {
-      invoker().invokeEverywhere(serializableRunnable());
-    }
-  }
-
-  @Override
-  protected void afterClass() throws Throwable {
-    if (this.disconnectAfterClass) {
-      invoker().invokeEverywhere(serializableRunnable());
-    }
-  }
-
   private static SerializableRunnable serializableRunnable() {
     return new SerializableRunnable() {
       @Override
       public void run() {
-        // TODO: disconnectFromDS();
+        // TODO:uncomment: disconnectFromDS();
       }
     };
   }
   
   /**
    * Builds an instance of DistributedDisconnectRule
-   * 
-   * @author Kirk Lund
    */
   public static class Builder {
-    private boolean disconnectBeforeClass;
-    private boolean disconnectAfterClass;
     private boolean disconnectBefore;
     private boolean disconnectAfter;
     
     public Builder() {}
 
-    public Builder disconnectBeforeClass(final boolean disconnectBeforeClass) {
-      this.disconnectBeforeClass = disconnectBeforeClass;
-      return this;
-    }
-    
     public Builder disconnectBefore(final boolean disconnectBefore) {
       this.disconnectBefore = disconnectBefore;
       return this;
     }
     
-    public Builder disconnectAfterClass(final boolean disconnectAfterClass) {
-      this.disconnectAfterClass = disconnectAfterClass;
-      return this;
-    }
-    
     public Builder disconnectAfter(final boolean disconnectAfter) {
       this.disconnectAfter = disconnectAfter;
       return this;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedExternalResource.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedExternalResource.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedExternalResource.java
index d3b7319..a7c3598 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedExternalResource.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedExternalResource.java
@@ -16,9 +16,12 @@
  */
 package com.gemstone.gemfire.test.dunit.rules;
 
-import com.gemstone.gemfire.test.junit.rules.SerializableExternalResource;
+import com.gemstone.gemfire.test.junit.rules.serializable.SerializableExternalResource;
 
-@SuppressWarnings("serial")
+/**
+ * Distributed version of SerializableExternalResource which affects all remote
+ * DUnit JVMs including the Locator JVM.
+ */
 public class DistributedExternalResource extends SerializableExternalResource {
 
   private final RemoteInvoker invoker;
@@ -35,24 +38,4 @@ public class DistributedExternalResource extends SerializableExternalResource {
   protected RemoteInvoker invoker() {
     return this.invoker;
   }
-  
-  @Override
-  protected void before() throws Throwable {
-    // do nothing
-  }
-
-  @Override
-  protected void after() throws Throwable {
-    // do nothing
-  }
-
-  @Override
-  protected void beforeClass() throws Throwable {
-    // do nothing
-  }
-
-  @Override
-  protected void afterClass() throws Throwable {
-    // do nothing
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
index 1711b21..7f95fad 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
@@ -24,15 +24,12 @@ import java.util.Properties;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-import com.gemstone.gemfire.test.junit.rules.SerializableTestRule;
+import com.gemstone.gemfire.test.junit.rules.serializable.SerializableTestRule;
 
 /**
  * Distributed version of RestoreSystemProperties which affects all DUnit 
  * JVMs including the Locator JVM.
- * 
- * @author Kirk Lund
  */
-@SuppressWarnings("serial")
 public class DistributedRestoreSystemProperties extends RestoreSystemProperties implements SerializableTestRule {
   
   private static volatile Properties originalProperties;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
index 9e3c5b2..98dbc2f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
@@ -23,8 +23,14 @@ import java.io.Serializable;
 
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 
-@SuppressWarnings("serial")
-public class RemoteInvoker implements Serializable {
+/**
+ * Provides remote invocation support to a {@code TestRule}. These methods
+ * will invoke a SerializableRunnable in all remote DUnit JVMs including the
+ * Locator JVM.
+ */
+class RemoteInvoker implements Serializable {
+
+  private static final long serialVersionUID = -1759722991299584649L;
 
   public void invokeEverywhere(final SerializableRunnable runnable) {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/build.gradle
----------------------------------------------------------------------
diff --git a/geode-junit/build.gradle b/geode-junit/build.gradle
index 112d112..7aa6562 100755
--- a/geode-junit/build.gradle
+++ b/geode-junit/build.gradle
@@ -16,6 +16,7 @@
  */
 
 dependencies {
+  testCompile 'commons-lang:commons-lang:' + project.'commons-lang.version'
   compile 'junit:junit:' + project.'junit.version'
   compile 'org.hamcrest:hamcrest-all:' + project.'hamcrest-all.version'
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/ConditionalIgnore.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/ConditionalIgnore.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/ConditionalIgnore.java
index b409cb1..5e037c7 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/ConditionalIgnore.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/ConditionalIgnore.java
@@ -29,7 +29,6 @@ import com.gemstone.gemfire.test.junit.support.DefaultIgnoreCondition;
  * conditionally ignore the test case for a fixed amount of time, or based on a predetermined condition provided by
  * the IgnoreCondition interface.
  *
- * @author John Blum
  * @see java.lang.annotation.Annotation
  * @see com.gemstone.gemfire.test.junit.IgnoreCondition
  * @see com.gemstone.gemfire.test.junit.support.DefaultIgnoreCondition

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreCondition.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreCondition.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreCondition.java
index 0caa959..15e507e 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreCondition.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreCondition.java
@@ -21,7 +21,6 @@ import org.junit.runner.Description;
 /**
  * The IgnoreCondition class...
  *
- * @author John Blum
  * @see org.junit.runner.Description
  */
 @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreUntil.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreUntil.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreUntil.java
index 5910d10..e60e92f 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreUntil.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/IgnoreUntil.java
@@ -29,7 +29,6 @@ import com.gemstone.gemfire.test.junit.support.DefaultIgnoreCondition;
  * conditionally ignore the test case for a fixed amount of time, or based on a predetermined condition provided by
  * the IgnoreCondition interface.
  *
- * @author John Blum
  * @see java.lang.annotation.Annotation
  * @see com.gemstone.gemfire.test.junit.IgnoreCondition
  * @see com.gemstone.gemfire.test.junit.support.DefaultIgnoreCondition

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Repeat.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Repeat.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Repeat.java
index 5cfa321..fc5c441 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Repeat.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Repeat.java
@@ -26,12 +26,11 @@ import java.lang.annotation.Target;
  * The Repeat class is a Java Annotation enabling an annotated test suite class test case method to be repeated
  * a specified number of iterations.
  *
- * @author John Blum
  * @see java.lang.annotation.Annotation
  */
 @Documented
 @Retention(RetentionPolicy.RUNTIME)
-@Target({ ElementType.METHOD })
+@Target(ElementType.METHOD)
 @SuppressWarnings("unused")
 public @interface Repeat {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Retry.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Retry.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Retry.java
index 65f3cf6..6039943 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Retry.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/Retry.java
@@ -23,8 +23,8 @@ import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
 /**
- * Java Annotation used to annotate a test suite class test case method in order to
- * retry it in case of failure up to the specified maximum attempts.
+ * Java Annotation used to annotate a test method in order to retry failures
+ * up to the specified maximum attempts. Default maximum attempts is one retry.
  */
 @Documented
 @Retention(RetentionPolicy.RUNTIME)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/ContainerTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/ContainerTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/ContainerTest.java
index 8eec738..7d593b5 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/ContainerTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/ContainerTest.java
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a test executes within a container
  * environment such as an OSGi server.
- *  
- * @author Kirk Lund
  */
 public interface ContainerTest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTest.java
old mode 100644
new mode 100755
index 6aa538b..b4ec17c
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTest.java
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a test involving multiple members of a
  * distributed system.
- * 
- * @author Kirk Lund
  */
 public interface DistributedTest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTransactionsTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTransactionsTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTransactionsTest.java
old mode 100644
new mode 100755
index 1892ddd..d9cb447
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTransactionsTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/DistributedTransactionsTest.java
@@ -17,9 +17,8 @@
 package com.gemstone.gemfire.test.junit.categories;
 
 /**
- * Unit test category that specifies tests for distributed transactions
+ * JUnit Test Category that specifies tests for distributed transactions
  * functionality
- * @author shirishd
  */
 public interface DistributedTransactionsTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/HydraTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/HydraTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/HydraTest.java
index 4fe535b..7293ed7 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/HydraTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/HydraTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a hydra test.
- *  
- * @author Kirk Lund
  */
 public interface HydraTest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/IntegrationTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/IntegrationTest.java
old mode 100644
new mode 100755
index 1be6e45..d2f8e04
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/IntegrationTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/IntegrationTest.java
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a test involving inter-operation of 
  * components or subsystems.
- *  
- * @author Kirk Lund
  */
 public interface IntegrationTest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/PerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/PerformanceTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/PerformanceTest.java
old mode 100644
new mode 100755
index ad952be..5785395
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/PerformanceTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/PerformanceTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a test that measures performance.
- *  
- * @author Kirk Lund
  */
 public interface PerformanceTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UITest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UITest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UITest.java
old mode 100644
new mode 100755
index 834931a..822272e
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UITest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UITest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a test involving a browser present to test the UI
- *  
- * @author Jinmei Liao
  */
 public interface UITest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UnitTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UnitTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UnitTest.java
old mode 100644
new mode 100755
index d5be9e8..3b81dd6
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UnitTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/UnitTest.java
@@ -15,11 +15,20 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
  * JUnit Test Category that specifies a test with very narrow and well defined
  * scope. Any complex dependencies and interactions are stubbed or mocked.
- *  
- * @author Kirk Lund
+ *
+ * <p><ul>A {@code UnitTest} should <bold>not<bold> do any of the following:
+ * <li>communicate with a database
+ * <li>communicate across the network
+ * <li>access the file system
+ * <li>prevent the running of other unit tests in parallel
+ * <li>require anything special in the environment (such as editing config files or running an external process)
+ * </ul>
+ *
+ * @see <a href="http://www.artima.com/weblogs/viewpost.jsp?thread=126923">A Set of Unit Testing Rules by Michael Feathers</a>
  */
 public interface UnitTest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/WanTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/WanTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/WanTest.java
old mode 100644
new mode 100755
index 9abf15b..6f75e87
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/WanTest.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/WanTest.java
@@ -15,10 +15,9 @@
  * limitations under the License.
  */
 package com.gemstone.gemfire.test.junit.categories;
+
 /**
- * Tests Wan Feature.
- * 
- * @author Kirk Lund 
+ * JUnit Test Category that specifies a test that involves Wan feature.
  */
 public interface WanTest {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ConditionalIgnoreRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ConditionalIgnoreRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ConditionalIgnoreRule.java
index 80898b5..6c354d2 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ConditionalIgnoreRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ConditionalIgnoreRule.java
@@ -33,7 +33,6 @@ import com.gemstone.gemfire.test.junit.support.IgnoreConditionEvaluationExceptio
 /**
  * The ConditionalIgnoreRule class...
  *
- * @author John Blum
  * @see org.junit.rules.TestRule
  * @see org.junit.runner.Description
  * @see org.junit.runners.model.Statement

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeout.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeout.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeout.java
deleted file mode 100755
index 2f53811..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeout.java
+++ /dev/null
@@ -1,180 +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.junit.rules;
-
-import static org.junit.Assert.assertThat;
-
-import java.util.concurrent.TimeUnit;
-
-import org.hamcrest.Matcher;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TestRule;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-/**
- * Expect an Exception within a specified timeout.
- * 
- * @author Kirk Lund
- * @since 8.2
- */
-public class ExpectedTimeout implements TestRule {
-
-  /**
-   * @return a Rule that expects no timeout (identical to behavior without this Rule)
-   */
-  public static ExpectedTimeout none() {
-    return new ExpectedTimeout();
-  }
-  
-  private ExpectedException delegate;
-  private boolean expectsThrowable;
-  private long minDuration;
-  private long maxDuration;
-  private TimeUnit timeUnit;
-  
-  private ExpectedTimeout() {
-    this.delegate = ExpectedException.none();
-  }
-
-  public ExpectedTimeout expectMinimumDuration(final long minDuration) {
-    this.minDuration = minDuration;
-    return this;
-  }
-  public ExpectedTimeout expectMaximumDuration(final long maxDuration) {
-    this.maxDuration = maxDuration;
-    return this;
-  }
-  public ExpectedTimeout expectTimeUnit(final TimeUnit timeUnit) {
-    this.timeUnit = timeUnit;
-    return this;
-  }
-
-  public ExpectedTimeout handleAssertionErrors() {
-    this.delegate.handleAssertionErrors();
-    return this;
-  }
-  
-  public ExpectedTimeout handleAssumptionViolatedExceptions() {
-    this.delegate.handleAssumptionViolatedExceptions();
-    return this;
-  }
-  
-  /**
-   * Adds {@code matcher} to the list of requirements for any thrown
-   * exception.
-   */
-  public void expect(final Matcher<?> matcher) {
-    this.delegate.expect(matcher);
-  }
-
-  /**
-   * Adds to the list of requirements for any thrown exception that it should
-   * be an instance of {@code type}
-   */
-  public void expect(final Class<? extends Throwable> type) {
-    this.delegate.expect(type);
-    this.expectsThrowable = true;
-  }
-
-  /**
-   * Adds to the list of requirements for any thrown exception that it should
-   * <em>contain</em> string {@code substring}
-   */
-  public void expectMessage(final String substring) {
-    this.delegate.expectMessage(substring);
-  }
-
-  /**
-   * Adds {@code matcher} to the list of requirements for the message returned
-   * from any thrown exception.
-   */
-  public void expectMessage(final Matcher<String> matcher) {
-    this.delegate.expectMessage(matcher);
-  }
-
-  /**
-   * Adds {@code matcher} to the list of requirements for the cause of
-   * any thrown exception.
-   */
-  public void expectCause(final Matcher<? extends Throwable> expectedCause) {
-    this.delegate.expectCause(expectedCause);
-  }
-
-  public boolean expectsTimeout() {
-    return minDuration > 0 || maxDuration > 0;
-  }
-  
-  public boolean expectsThrowable() {
-    return expectsThrowable = true;
-  }
-  
-  @Override
-  public Statement apply(final Statement base, final Description description) {
-    Statement next = delegate.apply(base, description);
-    return new ExpectedTimeoutStatement(next);
-  }
-  
-  private void handleTime(final Long duration) {
-    if (expectsTimeout()) {
-      assertThat(timeUnit.convert(duration, TimeUnit.NANOSECONDS), new TimeMatcher(timeUnit, minDuration, maxDuration));
-    }
-  }
-  
-  private static class TimeMatcher extends org.hamcrest.TypeSafeMatcher<Long> {
-    
-    private final TimeUnit timeUnit;
-    private final long minDuration;
-    private final long maxDuration;
- 
-    public TimeMatcher(final TimeUnit timeUnit, final long minDuration, final long maxDuration) {
-      this.timeUnit = timeUnit;
-      this.minDuration = minDuration;
-      this.maxDuration = maxDuration;
-    }
- 
-    @Override
-    public boolean matchesSafely(final Long duration) {
-      return duration >= this.minDuration && duration <= this.maxDuration;
-    }
-
-    @Override
-    public void describeTo(final org.hamcrest.Description description) {
-      description.appendText("expects duration to be greater than or equal to ")
-          .appendValue(this.minDuration)
-          .appendText(" and less than or equal to ")
-          .appendValue(this.maxDuration)
-          .appendText(" ")
-          .appendValue(this.timeUnit);
-    }
-  }
-  
-  private class ExpectedTimeoutStatement extends Statement {
-    private final Statement next;
-
-    public ExpectedTimeoutStatement(final Statement base) {
-      next = base;
-    }
-
-    @Override
-    public void evaluate() throws Throwable {
-      long start = System.nanoTime();
-      next.evaluate();
-      handleTime(System.nanoTime() - start);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
index 7b6d345..c7a98b8 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRule.java
@@ -28,9 +28,6 @@ import org.junit.runners.model.Statement;
 
 /**
  * Expect an Exception within a specified timeout.
- * 
- * @author Kirk Lund
- * @since 8.2
  */
 public class ExpectedTimeoutRule implements TestRule {
 
@@ -64,19 +61,8 @@ public class ExpectedTimeoutRule implements TestRule {
     return this;
   }
 
-  public ExpectedTimeoutRule handleAssertionErrors() {
-    this.delegate.handleAssertionErrors();
-    return this;
-  }
-  
-  public ExpectedTimeoutRule handleAssumptionViolatedExceptions() {
-    this.delegate.handleAssumptionViolatedExceptions();
-    return this;
-  }
-  
   /**
-   * Adds {@code matcher} to the list of requirements for any thrown
-   * exception.
+   * Adds {@code matcher} to the list of requirements for any thrown exception.
    */
   public void expect(final Matcher<?> matcher) {
     this.delegate.expect(matcher);
@@ -84,7 +70,7 @@ public class ExpectedTimeoutRule implements TestRule {
 
   /**
    * Adds to the list of requirements for any thrown exception that it should
-   * be an instance of {@code type}
+   * be an instance of {@code type}.
    */
   public void expect(final Class<? extends Throwable> type) {
     this.delegate.expect(type);
@@ -115,23 +101,29 @@ public class ExpectedTimeoutRule implements TestRule {
     this.delegate.expectCause(expectedCause);
   }
 
-  public boolean expectsTimeout() {
-    return minDuration > 0 || maxDuration > 0;
+  /**
+   * Returns true if a timeout is expected.
+   */
+  protected boolean expectsTimeout() {
+    return this.minDuration > 0 || this.maxDuration > 0;
   }
-  
-  public boolean expectsThrowable() {
-    return expectsThrowable = true;
+
+  /**
+   * Returns true if a Throwable is expected.
+   */
+  protected boolean expectsThrowable() {
+    return this.expectsThrowable;
   }
   
   @Override
   public Statement apply(final Statement base, final Description description) {
-    Statement next = delegate.apply(base, description);
+    Statement next = this.delegate.apply(base, description);
     return new ExpectedTimeoutStatement(next);
   }
   
   private void handleTime(final Long duration) {
     if (expectsTimeout()) {
-      assertThat(timeUnit.convert(duration, TimeUnit.NANOSECONDS), new TimeMatcher(timeUnit, minDuration, maxDuration));
+      assertThat(this.timeUnit.convert(duration, TimeUnit.NANOSECONDS), new TimeMatcher(this.timeUnit, this.minDuration, this.maxDuration));
     }
   }
   
@@ -167,13 +159,13 @@ public class ExpectedTimeoutRule implements TestRule {
     private final Statement next;
 
     public ExpectedTimeoutStatement(final Statement base) {
-      next = base;
+      this.next = base;
     }
 
     @Override
     public void evaluate() throws Throwable {
       long start = System.nanoTime();
-      next.evaluate();
+      this.next.evaluate();
       handleTime(System.nanoTime() - start);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRule.java
index bf4ec3f..f6ba368 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRule.java
@@ -33,7 +33,6 @@ import com.gemstone.gemfire.test.junit.support.IgnoreConditionEvaluationExceptio
 /**
  * The IgnoreUntilRule class...
  *
- * @author John Blum
  * @see org.junit.rules.TestRule
  * @see org.junit.runner.Description
  * @see org.junit.runners.model.Statement

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RepeatRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RepeatRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RepeatRule.java
index 7bfe538..8a45979 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RepeatRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RepeatRule.java
@@ -24,10 +24,9 @@ import org.junit.runners.model.Statement;
 import com.gemstone.gemfire.test.junit.Repeat;
 
 /**
- * The RepeatRule class is a JUnit TestRule that enables an appropriately @Repeat annotated test case method
+ * The RepeatRule class is a JUnit TestRule that enables an appropriately {@literal @}Repeat annotated test case method
  * to be repeated a specified number of times.
  *
- * @author John Blum
  * @see org.junit.rules.TestRule
  * @see org.junit.runner.Description
  * @see org.junit.runners.model.Statement

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
index c885aec..8535fa5 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
@@ -57,7 +57,6 @@ import com.gemstone.gemfire.test.junit.Retry;
  * This version of RetryRule will retry a test that fails because of any kind 
  * of Throwable.
  */
-@SuppressWarnings("serial")
 public class RetryRule implements TestRule, Serializable {
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RuleList.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RuleList.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RuleList.java
new file mode 100755
index 0000000..f353d5e
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RuleList.java
@@ -0,0 +1,95 @@
+/*
+ * 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.junit.rules;
+
+import org.junit.rules.TestRule;
+import org.junit.runner.Description;
+import org.junit.runners.model.Statement;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * The {@code RuleList} rule enables ordering of TestRules.
+ *
+ * <p>Example:
+ *
+ * <pre>
+ * public class SomeTest {
+ *
+ *   {@literal @}Rule
+ *   public RuleList rules = new RuleList().add(new FirstRule()
+ *                                         .add(new SecondRule()
+ *                                         .add(new ThirdRule();
+ * </pre>
+ */
+public class RuleList implements TestRule {
+  
+  private final List<TestRule> rules = new ArrayList<>();
+
+  /**
+   * Creates an empty {@code RuleList}.
+   */
+  public RuleList() {
+  }
+
+  /**
+   * Creates a {@code RuleList} containing a single {@link TestRule}.
+   *
+   * @param rule the first rule of the {@code RuleList}
+   */
+  public RuleList(final TestRule rule) {
+    this.rules.add(rule);
+  }
+
+  /**
+   * Creates a new {@code RuleList} containing the specified {@link TestRule}s.
+   *
+   * @param rules the list of {@code TestRule}s to add
+   */
+  protected RuleList(final List<TestRule> rules) {
+    this.rules.addAll(rules);
+  }
+
+  /**
+   * Adds a new {@code TestRule} to the end of the current {@code RuleList}.
+   *
+   * @param rule the rule to add.
+   * @return the {@code RuleList} with a new TestRule added
+   */
+  public RuleList add(final TestRule rule) {
+    this.rules.add(rule);
+    return this;
+  }
+
+  @Override
+  public Statement apply(Statement base, final Description description) {
+    for (TestRule each : this.rules) {
+      base = each.apply(base, description);
+    }
+    return base;
+  }
+
+  /**
+   * Returns a reference to the actual list of {@code TestRule}s. For use by
+   * subclasses and tests.
+   */
+  protected List<TestRule> rules() {
+    return this.rules;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableExternalResource.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableExternalResource.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableExternalResource.java
deleted file mode 100755
index 37d8eb5..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableExternalResource.java
+++ /dev/null
@@ -1,107 +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.junit.rules;
-
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-/**
- * Serializable version of ExternalResource JUnit Rule. JUnit lifecycle is not
- * executed in remote JVMs. The <tt>after()</tt> callback has a throws-clause 
- * that matches <tt>before()</tt>.
- * 
- * Implementation copied from <tt>org.junit.rules.ExternalResource</tt>.
- * 
- * @author Kirk Lund
- */
-@SuppressWarnings("serial")
-public abstract class SerializableExternalResource implements SerializableTestRule {
-  
-  @Override
-  public Statement apply(final Statement base, final Description description) {
-    if (description.isTest()) {
-      return statement(base);
-    } else if (description.isSuite()) {
-      return statementClass(base);
-    }
-    return base;
-  }
-
-  private Statement statement(final Statement base) {
-    return new Statement() {
-      @Override
-      public void evaluate() throws Throwable {
-        before();
-        try {
-          base.evaluate();
-        } finally {
-          after();
-        }
-      }
-    };
-  }
-
-  private Statement statementClass(final Statement base) {
-    return new Statement() {
-      @Override
-      public void evaluate() throws Throwable {
-        beforeClass();
-        try {
-          base.evaluate();
-        } finally {
-          afterClass();
-        }
-      }
-    };
-  }
-
-  /**
-   * Override to set up your specific external resource.
-   *
-   * @throws Throwable if setup fails (which will disable {@code after}
-   */
-  protected void before() throws Throwable {
-    // do nothing
-  }
-
-  /**
-   * Override to tear down your specific external resource.
-   * 
-   * @throws Throwable if teardown fails (which will disable {@code after}
-   */
-  protected void after() throws Throwable {
-    // do nothing
-  }
-
-  /**
-   * Override to set up your specific external resource.
-   *
-   * @throws Throwable if setup fails (which will disable {@code after}
-   */
-  protected void beforeClass() throws Throwable {
-    // do nothing
-  }
-
-  /**
-   * Override to tear down your specific external resource.
-   *
-   * @throws Throwable if teardown fails (which will disable {@code after}
-   */
-  protected void afterClass() throws Throwable {
-    // do nothing
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableRuleChain.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableRuleChain.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableRuleChain.java
deleted file mode 100755
index 936345e..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableRuleChain.java
+++ /dev/null
@@ -1,119 +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.junit.rules;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.junit.rules.TestRule;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-/**
- * Serializable version of TemporaryFolder JUnit Rule. JUnit lifecycle is not
- * executed in remote JVMs.
- * 
- * Implementation copied from <tt>org.junit.rules.RuleChain</tt>.
- * 
- * The SerializableRuleChain rule allows ordering of TestRules. You create a
- * {@code RuleChain} with {@link #outerRule(TestRule)} and subsequent calls of
- * {@link #around(TestRule)}:
- *
- * <pre>
- * public static class UseRuleChain {
- *  &#064;Rule
- *  public RuleChain chain= RuleChain
- *                         .outerRule(new LoggingRule("outer rule")
- *                         .around(new LoggingRule("middle rule")
- *                         .around(new LoggingRule("inner rule");
- *
- *  &#064;Test
- *  public void example() {
- *    assertTrue(true);
- *     }
- * }
- * </pre>
- *
- * writes the log
- *
- * <pre>
- * starting outer rule
- * starting middle rule
- * starting inner rule
- * finished inner rule
- * finished middle rule
- * finished outer rule
- * </pre>
- *
- * @author Kirk Lund
- */
-@SuppressWarnings("serial")
-public class SerializableRuleChain implements SerializableTestRule {
-  private static final SerializableRuleChain EMPTY_CHAIN = new SerializableRuleChain(Collections.<TestRule>emptyList());
-
-  private transient List<TestRule> rulesStartingWithInnerMost;
-
-  /**
-  * Returns a {@code SerializableRuleChain} without a {@link TestRule}. This method may
-  * be the starting point of a {@code SerializableRuleChain}.
-  *
-  * @return a {@code SerializableRuleChain} without a {@link TestRule}.
-  */
-  public static SerializableRuleChain emptyRuleChain() {
-    return EMPTY_CHAIN;
-  }
-
-  /**
-  * Returns a {@code SerializableRuleChain} with a single {@link TestRule}. This method
-  * is the usual starting point of a {@code SerializableRuleChain}.
-  *
-  * @param outerRule the outer rule of the {@code SerializableRuleChain}.
-  * @return a {@code SerializableRuleChain} with a single {@link TestRule}.
-  */
-  public static SerializableRuleChain outerRule(TestRule outerRule) {
-    return emptyRuleChain().around(outerRule);
-  }
-
-  private SerializableRuleChain(List<TestRule> rules) {
-    this.rulesStartingWithInnerMost = rules;
-  }
-
-  /**
-  * Create a new {@code SerializableRuleChain}, which encloses the {@code nextRule} with
-  * the rules of the current {@code SerializableRuleChain}.
-  *
-  * @param enclosedRule the rule to enclose.
-  * @return a new {@code SerializableRuleChain}.
-  */
-  public SerializableRuleChain around(TestRule enclosedRule) {
-    List<TestRule> rulesOfNewChain = new ArrayList<TestRule>();
-    rulesOfNewChain.add(enclosedRule);
-    rulesOfNewChain.addAll(rulesStartingWithInnerMost);
-    return new SerializableRuleChain(rulesOfNewChain);
-  }
-
-  /**
-  * {@inheritDoc}
-  */
-  public Statement apply(Statement base, Description description) {
-    for (TestRule each : rulesStartingWithInnerMost) {
-      base = each.apply(base, description);
-    }
-    return base;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTemporaryFolder.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTemporaryFolder.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTemporaryFolder.java
deleted file mode 100755
index 0e796b3..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTemporaryFolder.java
+++ /dev/null
@@ -1,70 +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.junit.rules;
-
-import java.io.File;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.lang.reflect.Field;
-
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Serializable version of TemporaryFolder JUnit Rule. JUnit lifecycle is not
- * executed in remote JVMs.
- * 
- * @author Kirk Lund
- */
-@SuppressWarnings("serial")
-public class SerializableTemporaryFolder extends TemporaryFolder implements SerializableTestRule {
-
-  private void writeObject(final ObjectOutputStream out) throws Exception {
-    writeParentFolder(out);
-    writeFolder(out);
-  }
-
-  private void readObject(final ObjectInputStream in) throws Exception {
-    readParentFolder(in);
-    readFolder(in);
-  }
-  
-  private void readParentFolder(final ObjectInputStream in) throws Exception {
-    final Field parentFolderField = TemporaryFolder.class.getDeclaredField("parentFolder");
-    parentFolderField.setAccessible(true);
-    parentFolderField.set(this, (File) in.readObject());
-  }
-  
-  private void readFolder(final ObjectInputStream in) throws Exception {
-    final Field folderField = TemporaryFolder.class.getDeclaredField("folder");
-    folderField.setAccessible(true);
-    folderField.set(this, (File) in.readObject());
-  }
-  
-  private void writeParentFolder(final ObjectOutputStream out) throws Exception {
-    final Field parentFolderField = TemporaryFolder.class.getDeclaredField("parentFolder");
-    parentFolderField.setAccessible(true);
-    final File parentFolderFieldValue = (File) parentFolderField.get(this);
-    out.writeObject(parentFolderFieldValue);
-  }
-  
-  private void writeFolder(final ObjectOutputStream out) throws Exception {
-    final Field folderField = TemporaryFolder.class.getDeclaredField("folder");
-    folderField.setAccessible(true);
-    final File folderFieldValue = (File) folderField.get(this);
-    out.writeObject(folderFieldValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestName.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestName.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestName.java
deleted file mode 100755
index 1fd255f..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestName.java
+++ /dev/null
@@ -1,54 +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.junit.rules;
-
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.lang.reflect.Field;
-
-import org.junit.rules.TestName;
-
-/**
- * Serializable version of TestName JUnit Rule. JUnit lifecycle is not
- * executed in remote JVMs.
- * 
- * @author Kirk Lund
- */
-@SuppressWarnings("serial")
-public class SerializableTestName extends TestName implements SerializableTestRule {
-
-  private void writeObject(final ObjectOutputStream out) throws Exception {
-    writeName(out);
-  }
-
-  private void readObject(final ObjectInputStream in) throws Exception {
-    readName(in);
-  }
-  
-  private void writeName(final ObjectOutputStream out) throws Exception {
-    final Field nameField = TestName.class.getDeclaredField("name");
-    nameField.setAccessible(true);
-    final String nameValue = (String) nameField.get(this);
-    out.writeObject(nameValue);
-  }
-  
-  private void readName(final ObjectInputStream in) throws Exception {
-    Field nameField = TestName.class.getDeclaredField("name");
-    nameField.setAccessible(true);
-    nameField.set(this, (String) in.readObject());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestRule.java
deleted file mode 100755
index 354c38a..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestRule.java
+++ /dev/null
@@ -1,33 +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.junit.rules;
-
-import java.io.Serializable;
-
-import org.junit.rules.TestRule;
-
-/**
- * Serializable version of JUnit TestRule. JUnit lifecycle is not
- * executed in remote JVMs.
- * 
- * The simplest way to satisfy this interface is to apply <tt>transient</tt>
- * to every instance field.
- * 
- * @author Kirk Lund
- */
-public interface SerializableTestRule extends Serializable, TestRule {
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestWatcher.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestWatcher.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestWatcher.java
deleted file mode 100755
index 5bcf686..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTestWatcher.java
+++ /dev/null
@@ -1,29 +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.junit.rules;
-
-import org.junit.rules.TestWatcher;
-
-/**
- * Serializable version of TestWatcher JUnit Rule. JUnit lifecycle is not
- * executed in remote JVMs.
- * 
- * @author Kirk Lund
- */
-@SuppressWarnings("serial")
-public class SerializableTestWatcher extends TestWatcher implements SerializableTestRule {
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTimeout.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTimeout.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTimeout.java
deleted file mode 100755
index 3136a1c..0000000
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/SerializableTimeout.java
+++ /dev/null
@@ -1,119 +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.junit.rules;
-
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.lang.reflect.Field;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
-import org.junit.rules.Timeout;
-
-/**
- * Serializable version of Timeout JUnit Rule. JUnit lifecycle is not
- * executed in remote JVMs.
- * 
- * @author Kirk Lund
- */
-@SuppressWarnings("serial")
-public class SerializableTimeout extends Timeout implements SerializableTestRule {
-
-  public static Builder builder() {
-    return new Builder();
-  }
-  
-  public SerializableTimeout(final long timeout, final TimeUnit timeUnit) {
-    super(timeout, timeUnit);
-  }
-  
-  protected SerializableTimeout(final Builder builder) {
-    super(builder);
-  }
-  
-  public static class Builder extends Timeout.Builder {
-    
-    protected Builder() {
-      super();
-    }
-    
-    @Override
-    public SerializableTimeout build() {
-      return new SerializableTimeout(this);
-    }
-  }
-
-  private void writeObject(final ObjectOutputStream out) throws Exception {
-    writeTimeout(out);
-    writeTimeUnit(out);
-    writeLookForStuckThread(out);
-  }
-
-  private void readObject(final ObjectInputStream in) throws Exception {
-    readTimeout(in);
-    readTimeUnit(in);
-    readLookForStuckThread(in);
-  }
-  
-  private void writeTimeout(final ObjectOutputStream out) throws Exception {
-    final Field timeoutField = TestName.class.getDeclaredField("timeout");
-    timeoutField.setAccessible(true);
-    final Long timeoutValue = (Long) timeoutField.get(this);
-    out.writeLong(timeoutValue);
-  }
-  
-  private void writeTimeUnit(final ObjectOutputStream out) throws Exception {
-    final Field timeoutField = TestName.class.getDeclaredField("timeUnit");
-    timeoutField.setAccessible(true);
-    final TimeUnit timeoutValue = (TimeUnit) timeoutField.get(this);
-    out.writeObject(timeoutValue);
-  }
-
-  private void writeLookForStuckThread(final ObjectOutputStream out) throws Exception {
-    try {
-      final Field lookForStuckThreadField = TemporaryFolder.class.getDeclaredField("lookForStuckThread");
-      lookForStuckThreadField.setAccessible(true);
-      final Boolean lookForStuckThreadValue = (Boolean) lookForStuckThreadField.get(this);
-      out.writeBoolean(lookForStuckThreadValue);
-    } catch (NoSuchFieldException e) {
-      out.writeBoolean(false);
-    }
-  }
-  
-  private void readTimeout(final ObjectInputStream in) throws Exception {
-    Field timeoutField = TestName.class.getDeclaredField("timeout");
-    timeoutField.setAccessible(true);
-    timeoutField.set(this, (Long) in.readObject());
-  }
-
-  private void readTimeUnit(final ObjectInputStream in) throws Exception {
-    Field timeUnitField = TestName.class.getDeclaredField("timeUnit");
-    timeUnitField.setAccessible(true);
-    timeUnitField.set(this, (TimeUnit) in.readObject());
-  }
-
-  private void readLookForStuckThread(final ObjectInputStream in) throws Exception {
-    try {
-      final Field lookForStuckThreadField = TemporaryFolder.class.getDeclaredField("lookForStuckThread");
-      lookForStuckThreadField.setAccessible(true);
-      lookForStuckThreadField.set(this, (Boolean) in.readObject());
-    } catch (NoSuchFieldException e) {
-      final boolean value = (Boolean) in.readObject();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldSerializationUtils.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldSerializationUtils.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldSerializationUtils.java
new file mode 100755
index 0000000..97e9265
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldSerializationUtils.java
@@ -0,0 +1,48 @@
+/*
+ * 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.junit.rules.serializable;
+
+import java.lang.reflect.Field;
+
+/**
+ * Provides support for serialization of private fields by reflection.
+ */
+public class FieldSerializationUtils {
+
+  protected FieldSerializationUtils() {
+  }
+
+  public static Object readField(final Class targetClass, final Object targetInstance, final String fieldName) {
+    try {
+      Field assureDeletionField = targetClass.getDeclaredField(fieldName);
+      assureDeletionField.setAccessible(true);
+      return assureDeletionField.get(targetInstance);
+    } catch (IllegalAccessException | NoSuchFieldException e) {
+      throw new Error(e);
+    }
+  }
+
+  public static void writeField(final Class targetClass, final Object targetInstance, final String fieldName, final Object value) {
+    try {
+      Field folderField = targetClass.getDeclaredField(fieldName);
+      folderField.setAccessible(true);
+      folderField.set(targetInstance, value);
+    } catch (IllegalAccessException | NoSuchFieldException e) {
+      throw new Error(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTemporaryFolder.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTemporaryFolder.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTemporaryFolder.java
new file mode 100755
index 0000000..3dd5ebd
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTemporaryFolder.java
@@ -0,0 +1,26 @@
+/*
+ * 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.junit.rules.serializable;
+
+/**
+ * Names of member fields in {@link org.junit.rules.TemporaryFolder}.
+ */
+interface FieldsOfTemporaryFolder {
+  static final String FIELD_PARENT_FOLDER = "parentFolder"; // java.io.File
+  static final String FIELD_ASSURE_DELETION = "assureDeletion"; // boolean (since JUnit 4.13)
+  static final String FIELD_FOLDER = "folder"; // java.io.File
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTestName.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTestName.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTestName.java
new file mode 100755
index 0000000..f30ed10
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTestName.java
@@ -0,0 +1,24 @@
+/*
+ * 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.junit.rules.serializable;
+
+/**
+ * Names of member fields in {@link org.junit.rules.TestName}.
+ */
+interface FieldsOfTestName {
+  static final String FIELD_NAME = "name"; // String
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTimeout.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTimeout.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTimeout.java
new file mode 100755
index 0000000..fab28ed
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/FieldsOfTimeout.java
@@ -0,0 +1,26 @@
+/*
+ * 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.junit.rules.serializable;
+
+/**
+ * Names of member fields in {@link org.junit.rules.Timeout}.
+ */
+interface FieldsOfTimeout {
+  static final String FIELD_TIMEOUT = "timeout"; // long
+  static final String FIELD_TIME_UNIT = "timeUnit"; // java.util.concurrent.TimeUnit
+  static final String FIELD_LOOK_FOR_STUCK_THREAD = "lookForStuckThread"; // boolean
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResource.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResource.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResource.java
new file mode 100755
index 0000000..772891f
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableExternalResource.java
@@ -0,0 +1,25 @@
+/*
+ * 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.junit.rules.serializable;
+
+import org.junit.rules.ExternalResource;
+
+/**
+ * Serializable subclass of {@link org.junit.rules.ExternalResource ExternalResource}.
+ */
+public abstract class SerializableExternalResource extends ExternalResource implements SerializableTestRule {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleList.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleList.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleList.java
new file mode 100755
index 0000000..3ae3079
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableRuleList.java
@@ -0,0 +1,78 @@
+/*
+ * 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.junit.rules.serializable;
+
+import com.gemstone.gemfire.test.junit.rules.RuleList;
+import org.junit.rules.TestRule;
+
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Serializable version of {@link RuleList}.
+ */
+public class SerializableRuleList extends RuleList implements SerializableTestRule {
+
+  public SerializableRuleList() {
+    super();
+  }
+
+  public SerializableRuleList(final TestRule rule) {
+    super(rule);
+  }
+
+  protected SerializableRuleList(final List<TestRule> rules) {
+    super(rules);
+  }
+
+  @Override
+  public SerializableRuleList add(final TestRule rule) {
+    super.add(rule);
+    return this;
+  }
+
+  @Override
+  protected List<TestRule> rules() {
+    return super.rules();
+  }
+
+  private void readObject(final ObjectInputStream stream) throws InvalidObjectException {
+    throw new InvalidObjectException("SerializationProxy required");
+  }
+
+  private Object writeReplace() {
+    return new SerializationProxy(this);
+  }
+
+  /**
+   * Serialization proxy for {@code SerializableRuleList}.
+   */
+  private static class SerializationProxy implements Serializable {
+
+    private final List<TestRule> rules;
+
+    SerializationProxy(final SerializableRuleList instance) {
+      this.rules = instance.rules();
+    }
+
+    private Object readResolve() {
+      return new SerializableRuleList(this.rules);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolder.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolder.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolder.java
new file mode 100755
index 0000000..390cdc7
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTemporaryFolder.java
@@ -0,0 +1,70 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldSerializationUtils.*;
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldsOfTemporaryFolder.*;
+
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+/**
+ * Serializable subclass of {@link org.junit.rules.TemporaryFolder TemporaryFolder}.
+ * Instance variables of TemporaryFolder are serialized by reflection.
+ */
+public class SerializableTemporaryFolder extends TemporaryFolder implements SerializableTestRule {
+
+  public SerializableTemporaryFolder() {
+    super();
+  }
+
+  public SerializableTemporaryFolder(final File parentFolder) {
+    super(parentFolder);
+  }
+
+  private void readObject(final ObjectInputStream stream) throws InvalidObjectException {
+    throw new InvalidObjectException("SerializationProxy required");
+  }
+
+  private Object writeReplace() {
+    return new SerializationProxy(this);
+  }
+
+  /**
+   * Serialization proxy for {@code SerializableTemporaryFolder}.
+   */
+  private static class SerializationProxy implements Serializable {
+
+    private final File parentFolder;
+    private final File folder;
+
+    SerializationProxy(final SerializableTemporaryFolder instance) {
+      this.parentFolder = (File) readField(TemporaryFolder.class, instance, FIELD_PARENT_FOLDER);
+      this.folder =(File) readField(TemporaryFolder.class, instance, FIELD_FOLDER);
+    }
+
+    private Object readResolve() {
+      SerializableTemporaryFolder instance = new SerializableTemporaryFolder(this.parentFolder);
+      writeField(TemporaryFolder.class, instance, FIELD_FOLDER, this.folder);
+      return instance;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestName.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestName.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestName.java
new file mode 100755
index 0000000..cb87882
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestName.java
@@ -0,0 +1,65 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldSerializationUtils.*;
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldsOfTestName.*;
+
+import org.junit.rules.TestName;
+import org.junit.runner.Description;
+
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+
+/**
+ * Serializable subclass of {@link org.junit.rules.TestName TestName}. All
+ * instance variables of {@code TestName} are serialized by reflection.
+ */
+public class SerializableTestName extends TestName implements SerializableTestRule {
+
+  @Override
+  protected void starting(final Description description) {
+    super.starting(description);
+  }
+
+  private void readObject(final ObjectInputStream stream) throws InvalidObjectException {
+    throw new InvalidObjectException("SerializationProxy required");
+  }
+
+  private Object writeReplace() {
+    return new SerializationProxy(this);
+  }
+
+  /**
+   * Serialization proxy for {@code SerializableTestName}.
+   */
+  private static class SerializationProxy implements Serializable {
+
+    private final String name;
+
+    SerializationProxy(final SerializableTestName instance) {
+      this.name = (String) readField(TestName.class, instance, FIELD_NAME);
+    }
+
+    private Object readResolve() {
+      SerializableTestName instance = new SerializableTestName();
+      writeField(TestName.class, instance, FIELD_NAME, this.name);
+      return instance;
+    }
+  }
+}



[4/7] incubator-geode git commit: GEODE-953: Cleanup geode-junit

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestRule.java
new file mode 100755
index 0000000..9d06bc6
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestRule.java
@@ -0,0 +1,28 @@
+/*
+ * 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.junit.rules.serializable;
+
+import java.io.Serializable;
+
+import org.junit.rules.TestRule;
+
+/**
+ * Specifies that a {@link org.junit.rules.TestRule TestRule} is
+ * {@code Serializable}.
+ */
+public interface SerializableTestRule extends Serializable, TestRule {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcher.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcher.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcher.java
new file mode 100755
index 0000000..d325b09
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTestWatcher.java
@@ -0,0 +1,26 @@
+/*
+ * 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.junit.rules.serializable;
+
+import org.junit.rules.TestWatcher;
+
+/**
+ * Serializable subclass of {@link org.junit.rules.TestWatcher TestWatcher}.
+ * There are no instance variables to be serialized.
+ */
+public abstract class SerializableTestWatcher extends TestWatcher implements SerializableTestRule {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeout.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeout.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeout.java
new file mode 100755
index 0000000..f3a4ba7
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/serializable/SerializableTimeout.java
@@ -0,0 +1,104 @@
+/*
+ * 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.junit.rules.serializable;
+
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldSerializationUtils.*;
+import static com.gemstone.gemfire.test.junit.rules.serializable.FieldsOfTimeout.*;
+
+import org.junit.rules.Timeout;
+
+import java.io.InvalidObjectException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Serializable subclass of {@link org.junit.rules.Timeout Timeout}. All
+ * instance variables of {@code Timeout} are serialized by reflection.
+ */
+public class SerializableTimeout extends Timeout implements SerializableTestRule {
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public SerializableTimeout(final long timeout, final TimeUnit timeUnit) {
+    super(timeout, timeUnit);
+  }
+
+  protected SerializableTimeout(final Builder builder) {
+    super(builder);
+  }
+
+  private void readObject(final ObjectInputStream stream) throws InvalidObjectException {
+    throw new InvalidObjectException("SerializationProxy required");
+  }
+
+  private Object writeReplace() {
+    return new SerializationProxy(this);
+  }
+
+  /**
+   * Builder for {@code SerializableTimeout}.
+   */
+  public static class Builder extends Timeout.Builder {
+    
+    protected Builder() {
+      super();
+    }
+
+    @Override
+    public Builder withTimeout(final long timeout, final TimeUnit unit) {
+      super.withTimeout(timeout, unit);
+      return this;
+    }
+
+    @Override
+    public Builder withLookingForStuckThread(final boolean enable) {
+      super.withLookingForStuckThread(enable);
+      return this;
+    }
+
+    @Override
+    public SerializableTimeout build() {
+      return new SerializableTimeout(this);
+    }
+  }
+
+  /**
+   * Serialization proxy for {@code SerializableTimeout}.
+   */
+  private static class SerializationProxy implements Serializable {
+
+    private final long timeout;
+    private final TimeUnit timeUnit;
+    private final boolean lookForStuckThread;
+
+    SerializationProxy(final SerializableTimeout instance) {
+      this.timeout = (long) readField(Timeout.class, instance, FIELD_TIMEOUT);
+      this.timeUnit =(TimeUnit) readField(Timeout.class, instance, FIELD_TIME_UNIT);
+      this.lookForStuckThread =(boolean) readField(Timeout.class, instance, FIELD_LOOK_FOR_STUCK_THREAD);
+    }
+
+    private Object readResolve() {
+      return new SerializableTimeout.Builder()
+          .withTimeout(this.timeout, this.timeUnit)
+          .withLookingForStuckThread(this.lookForStuckThread)
+          .build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/DefaultIgnoreCondition.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/DefaultIgnoreCondition.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/DefaultIgnoreCondition.java
index b721c41..0f30c80 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/DefaultIgnoreCondition.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/DefaultIgnoreCondition.java
@@ -22,7 +22,6 @@ import com.gemstone.gemfire.test.junit.IgnoreCondition;
 /**
  * The DefaultIgnoreCondition class...
  *
- * @author John Blum
  * @see org.junit.runner.Description
  * @see com.gemstone.gemfire.test.junit.ConditionalIgnore
  * @see com.gemstone.gemfire.test.junit.IgnoreCondition
@@ -46,7 +45,7 @@ public class DefaultIgnoreCondition implements IgnoreCondition {
   }
 
   public boolean isIgnore() {
-    return ignore;
+    return this.ignore;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/IgnoreConditionEvaluationException.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/IgnoreConditionEvaluationException.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/IgnoreConditionEvaluationException.java
index 0d8a321..958e371 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/IgnoreConditionEvaluationException.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/support/IgnoreConditionEvaluationException.java
@@ -19,7 +19,6 @@ package com.gemstone.gemfire.test.junit.support;
 /**
  * The IgnoreConditionEvaluationException class...
  *
- * @author John Blum
  * @see java.lang.RuntimeException
  */
 @SuppressWarnings({ "serial", "unused" })

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutJUnitTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutJUnitTest.java
deleted file mode 100755
index 24ebdde..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutJUnitTest.java
+++ /dev/null
@@ -1,204 +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.junit.rules;
-
-import static org.hamcrest.core.StringContains.*;
-import static org.hamcrest.core.Is.*;
-import static org.hamcrest.core.IsInstanceOf.*;
-import static org.junit.Assert.*;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.JUnitCore;
-import org.junit.runner.Request;
-import org.junit.runner.Result;
-import org.junit.runner.notification.Failure;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- * Unit tests for ExpectedTimeout JUnit Rule.
- * 
- * @author Kirk Lund
- * @since 8.2
- */
-@Category(UnitTest.class)
-public class ExpectedTimeoutJUnitTest {
-
-  @Test
-  public void passesUnused() {
-    Result result = runTest(PassesUnused.class);
-    assertTrue(result.wasSuccessful());
-  }
-  
-  @Test
-  public void failsWithoutExpectedException() {
-    Result result = runTest(FailsWithoutExpectedException.class);
-    assertFalse(result.wasSuccessful());
-    List<Failure> failures = result.getFailures();
-    assertEquals(1, failures.size());
-    Failure failure = failures.get(0);
-    assertThat(failure.getException(), is(instanceOf(AssertionError.class)));
-    assertThat(failure.getException().getMessage(), containsString("Expected test to throw an instance of " + TimeoutException.class.getName()));
-  }
-  
-  @Test
-  public void failsWithoutExpectedTimeoutException() {
-    Result result = runTest(FailsWithoutExpectedTimeoutException.class);
-    assertFalse(result.wasSuccessful());
-    List<Failure> failures = result.getFailures();
-    assertEquals(1, failures.size());
-    Failure failure = failures.get(0);
-    assertThat(failure.getException(), is(instanceOf(AssertionError.class)));
-    assertThat(failure.getException().getMessage(), containsString("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWithoutExpectedTimeoutException.message + "\")"));
-  }
-  
-  @Test
-  public void failsWithExpectedTimeoutButWrongError() {
-    Result result = runTest(FailsWithExpectedTimeoutButWrongError.class);
-    assertFalse(result.wasSuccessful());
-    List<Failure> failures = result.getFailures();
-    assertEquals(1, failures.size());
-    Failure failure = failures.get(0);
-    assertThat(failure.getException(), is(instanceOf(AssertionError.class)));
-    assertThat(failure.getException().getMessage(), containsString(NullPointerException.class.getName()));
-  }
-  
-  @Test
-  public void passesWithExpectedTimeoutAndTimeoutException() {
-    Result result = runTest(PassesWithExpectedTimeoutAndTimeoutException.class);
-    assertTrue(result.wasSuccessful());
-  }
-  
-  @Test
-  public void failsWhenTimeoutIsEarly() {
-    Result result = runTest(FailsWhenTimeoutIsEarly.class);
-    assertFalse(result.wasSuccessful());
-    List<Failure> failures = result.getFailures();
-    assertEquals(1, failures.size());
-    Failure failure = failures.get(0);
-    assertThat(failure.getException(), is(instanceOf(AssertionError.class)));
-    assertThat(failure.getException().getMessage(), containsString("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsEarly.message + "\")"));
-  }
-  
-  @Test
-  public void failsWhenTimeoutIsLate() {
-    Result result = runTest(FailsWhenTimeoutIsLate.class);
-    assertFalse(result.wasSuccessful());
-    List<Failure> failures = result.getFailures();
-    assertEquals(1, failures.size());
-    Failure failure = failures.get(0);
-    assertThat(failure.getException(), is(instanceOf(AssertionError.class)));
-    assertThat(failure.getException().getMessage(), containsString("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsLate.message + "\")"));
-  }
-  
-  private static Result runTest(Class<?> test) {
-    JUnitCore junitCore = new JUnitCore();
-    return junitCore.run(Request.aClass(test).getRunner());
-  }
-  
-  public static class AbstractExpectedTimeoutTest {
-    @Rule
-    public ExpectedTimeout timeout = ExpectedTimeout.none();
-  }
-  
-  public static class PassesUnused extends AbstractExpectedTimeoutTest {
-    @Test
-    public void passesUnused() throws Exception {
-    }
-  }
-  
-  public static class FailsWithoutExpectedException extends AbstractExpectedTimeoutTest {
-    @Test
-    public void failsWithoutExpectedException() throws Exception {
-      timeout.expect(TimeoutException.class);
-    }
-  }
-  
-  public static class FailsWithoutExpectedTimeoutException extends AbstractExpectedTimeoutTest {
-    public static final String message = "this is a message for FailsWithoutExpectedTimeoutException";
-    @Test
-    public void failsWithoutExpectedTimeoutAndTimeoutException() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(1000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-    }
-  }
-  
-  public static class FailsWithExpectedTimeoutButWrongError extends AbstractExpectedTimeoutTest {
-    public static final String message = "this is a message for FailsWithExpectedTimeoutButWrongError";
-    @Test
-    public void failsWithExpectedTimeoutButWrongError() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(1000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-      throw new NullPointerException();
-    }
-  }
-
-  public static class PassesWithExpectedTimeoutAndTimeoutException extends AbstractExpectedTimeoutTest {
-    public static final String message = "this is a message for PassesWithExpectedTimeoutAndTimeoutException";
-    public static final Class<TimeoutException> exceptionClass = TimeoutException.class;
-    @Test
-    public void passesWithExpectedTimeoutAndTimeoutException() throws Exception {
-      timeout.expect(exceptionClass);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(1000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-      throw new TimeoutException(message);
-    }
-  }
-
-  public static class FailsWhenTimeoutIsEarly extends AbstractExpectedTimeoutTest {
-    public static final String message = "this is a message for FailsWhenTimeoutIsEarly";
-    @Test
-    public void failsWhenTimeoutIsEarly() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(1000);
-      timeout.expectMaximumDuration(2000);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(10);
-    }
-  }
-
-  public static class FailsWhenTimeoutIsLate extends AbstractExpectedTimeoutTest {
-    public static final String message = "this is a message for FailsWhenTimeoutIsLate";
-    @Test
-    public void failsWhenTimeoutIsLate() throws Exception {
-      timeout.expect(TimeoutException.class);
-      timeout.expectMessage(message);
-      timeout.expectMinimumDuration(10);
-      timeout.expectMaximumDuration(20);
-      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
-      Thread.sleep(100);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRuleTest.java
new file mode 100755
index 0000000..c1ad074
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/ExpectedTimeoutRuleTest.java
@@ -0,0 +1,246 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link ExpectedTimeoutRule}.
+ */
+@Category(UnitTest.class)
+public class ExpectedTimeoutRuleTest {
+
+  @Test
+  public void passesUnused() {
+    Result result = TestRunner.runTest(PassingTestShouldPassWhenUnused.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+  }
+  
+  @Test
+  public void failsWithoutExpectedException() {
+    Result result = TestRunner.runTest(FailsWithoutExpectedException.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+    
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw an instance of " + TimeoutException.class.getName());
+  }
+  
+  @Test
+  public void failsWithoutExpectedTimeoutException() {
+    Result result = TestRunner.runTest(FailsWithoutExpectedTimeoutException.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+    
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWithoutExpectedTimeoutException.message + "\")");
+  }
+  
+  @Test
+  public void failsWithExpectedTimeoutButWrongError() {
+    Result result = TestRunner.runTest(FailsWithExpectedTimeoutButWrongError.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+    
+    Failure failure = failures.get(0);
+    String expectedMessage = 
+        "\n" + 
+        "Expected: (an instance of java.util.concurrent.TimeoutException and exception with message a string containing \"this is a message for FailsWithExpectedTimeoutButWrongError\")" +
+        "\n" + 
+        "     " +
+        "but: an instance of java.util.concurrent.TimeoutException <java.lang.NullPointerException> is a java.lang.NullPointerException";
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessageContaining(expectedMessage);
+  }
+  
+  @Test
+  public void passesWithExpectedTimeoutAndTimeoutException() {
+    Result result = TestRunner.runTest(PassesWithExpectedTimeoutAndTimeoutException.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+  }
+  
+  @Test
+  public void failsWhenTimeoutIsEarly() {
+    Result result = TestRunner.runTest(FailsWhenTimeoutIsEarly.class);
+   
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+    
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsEarly.message + "\")");
+  }
+  
+  @Test
+  public void failsWhenTimeoutIsLate() {
+    Result result = TestRunner.runTest(FailsWhenTimeoutIsLate.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+    
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage("Expected test to throw (an instance of " + TimeoutException.class.getName() + " and exception with message a string containing \"" + FailsWhenTimeoutIsLate.message + "\")");
+  }
+
+  /**
+   * Base class for all inner class test cases
+   */
+  public static class AbstractExpectedTimeoutRuleTest {
+
+    @Rule
+    public ExpectedTimeoutRule timeout = ExpectedTimeoutRule.none();
+  }
+
+  /**
+   * Used by test {@link #passesUnused()}
+   */
+  public static class PassingTestShouldPassWhenUnused extends AbstractExpectedTimeoutRuleTest {
+
+    @Test
+    public void doTest() {
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWithoutExpectedException()}
+   */
+  public static class FailsWithoutExpectedException extends AbstractExpectedTimeoutRuleTest {
+
+    @Test
+    public void doTest() {
+      timeout.expect(TimeoutException.class);
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWithoutExpectedTimeoutException()}
+   */
+  public static class FailsWithoutExpectedTimeoutException extends AbstractExpectedTimeoutRuleTest {
+
+    static final String message = "this is a message for FailsWithoutExpectedTimeoutException";
+
+    @Test
+    public void doTest() throws Exception {
+      timeout.expect(TimeoutException.class);
+      timeout.expectMessage(message);
+      timeout.expectMinimumDuration(10);
+      timeout.expectMaximumDuration(1000);
+      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWithExpectedTimeoutButWrongError()}
+   */
+  public static class FailsWithExpectedTimeoutButWrongError extends AbstractExpectedTimeoutRuleTest {
+
+    static final String message = "this is a message for FailsWithExpectedTimeoutButWrongError";
+
+    @Test
+    public void doTest() throws Exception {
+      timeout.expect(TimeoutException.class);
+      timeout.expectMessage(message);
+      timeout.expectMinimumDuration(10);
+      timeout.expectMaximumDuration(1000);
+      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
+      throw new NullPointerException();
+    }
+  }
+
+  /**
+   * Used by test {@link #passesWithExpectedTimeoutAndTimeoutException()}
+   */
+  public static class PassesWithExpectedTimeoutAndTimeoutException extends AbstractExpectedTimeoutRuleTest {
+
+    static final String message = "this is a message for PassesWithExpectedTimeoutAndTimeoutException";
+    static final Class<TimeoutException> exceptionClass = TimeoutException.class;
+
+    @Test
+    public void doTest() throws Exception {
+      timeout.expect(exceptionClass);
+      timeout.expectMessage(message);
+      timeout.expectMinimumDuration(10);
+      timeout.expectMaximumDuration(1000);
+      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
+      throw new TimeoutException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWhenTimeoutIsEarly()}
+   */
+  public static class FailsWhenTimeoutIsEarly extends AbstractExpectedTimeoutRuleTest {
+
+    static final String message = "this is a message for FailsWhenTimeoutIsEarly";
+
+    @Test
+    public void doTest() throws Exception {
+      timeout.expect(TimeoutException.class);
+      timeout.expectMessage(message);
+      timeout.expectMinimumDuration(1000);
+      timeout.expectMaximumDuration(2000);
+      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
+      Thread.sleep(10);
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWhenTimeoutIsLate()}
+   */
+  public static class FailsWhenTimeoutIsLate extends AbstractExpectedTimeoutRuleTest {
+
+    static final String message = "this is a message for FailsWhenTimeoutIsLate";
+
+    @Test
+    public void doTest() throws Exception {
+      timeout.expect(TimeoutException.class);
+      timeout.expectMessage(message);
+      timeout.expectMinimumDuration(10);
+      timeout.expectMaximumDuration(20);
+      timeout.expectTimeUnit(TimeUnit.MILLISECONDS);
+      Thread.sleep(100);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRuleTest.java
new file mode 100755
index 0000000..d080f7c
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/IgnoreUntilRuleTest.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.test.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.IgnoreUntil;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link IgnoreUntilRule}.
+ */
+@Category(UnitTest.class)
+public class IgnoreUntilRuleTest {
+
+  private static final String ASSERTION_ERROR_MESSAGE = "failing test";
+  
+  @Test
+  public void shouldIgnoreWhenUntilIsInFuture() {
+    Result result = TestRunner.runTest(ShouldIgnoreWhenUntilIsInFuture.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(ShouldIgnoreWhenUntilIsInFuture.count).isEqualTo(0);
+  }
+  
+  @Test
+  public void shouldExecuteWhenUntilIsInPast() {
+    Result result = TestRunner.runTest(ShouldExecuteWhenUntilIsInPast.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
+    assertThat(ShouldExecuteWhenUntilIsInPast.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void shouldExecuteWhenUntilIsDefault() {
+    Result result = TestRunner.runTest(ShouldExecuteWhenUntilIsDefault.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
+    assertThat(ShouldExecuteWhenUntilIsDefault.count).isEqualTo(1);
+  }
+
+  /**
+   * Used by test {@link #shouldIgnoreWhenUntilIsInFuture()}
+   */
+  public static class ShouldIgnoreWhenUntilIsInFuture {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public final IgnoreUntilRule ignoreUntilRule = new IgnoreUntilRule();
+
+    @Test
+    @IgnoreUntil(value = "description", until = "3000-01-01")
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+
+  /**
+   * Used by test {@link #shouldExecuteWhenUntilIsInPast()}
+   */
+  public static class ShouldExecuteWhenUntilIsInPast {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public final IgnoreUntilRule ignoreUntilRule = new IgnoreUntilRule();
+
+    @Test
+    @IgnoreUntil(value = "description", until = "1980-01-01")
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+
+  /**
+   * Used by test {@link #shouldExecuteWhenUntilIsDefault()}
+   */
+  public static class ShouldExecuteWhenUntilIsDefault {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public final IgnoreUntilRule ignoreUntilRule = new IgnoreUntilRule();
+
+    @Test
+    @IgnoreUntil("description")
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RepeatRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RepeatRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RepeatRuleTest.java
new file mode 100755
index 0000000..235b16f
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RepeatRuleTest.java
@@ -0,0 +1,411 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.Repeat;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link RepeatRule}.
+ */
+@Category(UnitTest.class)
+public class RepeatRuleTest {
+
+  private static final String ASSERTION_ERROR_MESSAGE = "failing test";
+  
+  @Test
+  public void failingTestShouldFailOneTimeWhenRepeatIsUnused() {
+    Result result = TestRunner.runTest(FailingTestShouldFailOneTimeWhenRepeatIsUnused.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
+    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsUnused.count).isEqualTo(1);
+  }
+
+  @Test
+  public void passingTestShouldPassOneTimeWhenRepeatIsUnused() {
+    Result result = TestRunner.runTest(PassingTestShouldPassOneTimeWhenRepeatIsUnused.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassingTestShouldPassOneTimeWhenRepeatIsUnused.count).isEqualTo(1);
+  }
+
+  @Test
+  public void zeroValueShouldThrowIllegalArgumentException() {
+    Result result = TestRunner.runTest(ZeroValueShouldThrowIllegalArgumentException.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage("Repeat value must be a positive integer");
+    assertThat(ZeroValueShouldThrowIllegalArgumentException.count).isEqualTo(0);
+  }
+  
+  @Test
+  public void negativeValueShouldThrowIllegalArgumentException() {
+    Result result = TestRunner.runTest(NegativeValueShouldThrowIllegalArgumentException.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage("Repeat value must be a positive integer");
+    assertThat(NegativeValueShouldThrowIllegalArgumentException.count).isEqualTo(0);
+  }
+
+  /**
+   * Characterizes the behavior but is not a requirement for {@code RepeatRule}.
+   */
+  @Test
+  public void passingTestShouldBeSkippedWhenRepeatIsZero() {
+    Result result = TestRunner.runTest(PassingTestShouldBeSkippedWhenRepeatIsZero.class);
+
+    assertThat(result.wasSuccessful()).isFalse();
+    assertThat(PassingTestShouldBeSkippedWhenRepeatIsZero.count).isEqualTo(0);
+  }
+
+  @Test
+  public void failingTestShouldFailOneTimeWhenRepeatIsOne() {
+    Result result = TestRunner.runTest(FailingTestShouldFailOneTimeWhenRepeatIsOne.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
+    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsOne.count).isEqualTo(1);
+  }
+
+  @Test
+  public void passingTestShouldPassOneTimeWhenRepeatIsOne() {
+    Result result = TestRunner.runTest(PassingTestShouldPassOneTimeWhenRepeatIsOne.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassingTestShouldPassOneTimeWhenRepeatIsOne.count).isEqualTo(1);
+  }
+
+  @Test
+  public void failingTestShouldFailOneTimeWhenRepeatIsTwo() {
+    Result result = TestRunner.runTest(FailingTestShouldFailOneTimeWhenRepeatIsTwo.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
+    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsTwo.count).isEqualTo(1);
+  }
+
+  @Test
+  public void passingTestShouldPassTwoTimesWhenRepeatIsTwo() {
+    Result result = TestRunner.runTest(PassingTestShouldPassTwoTimesWhenRepeatIsTwo.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassingTestShouldPassTwoTimesWhenRepeatIsTwo.count).isEqualTo(2);
+  }
+
+  @Test
+  public void failingTestShouldFailOneTimeWhenRepeatIsThree() {
+    Result result = TestRunner.runTest(FailingTestShouldFailOneTimeWhenRepeatIsThree.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(ASSERTION_ERROR_MESSAGE);
+    assertThat(FailingTestShouldFailOneTimeWhenRepeatIsThree.count).isEqualTo(1);
+  }
+
+  @Test
+  public void passingTestShouldPassThreeTimesWhenRepeatIsThree() {
+    Result result = TestRunner.runTest(PassingTestShouldPassThreeTimesWhenRepeatIsThree.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassingTestShouldPassThreeTimesWhenRepeatIsThree.count).isEqualTo(3);
+  }
+
+  /**
+   * Used by test {@link #failingTestShouldFailOneTimeWhenRepeatIsUnused()}
+   */
+  public static class FailingTestShouldFailOneTimeWhenRepeatIsUnused {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+
+  /**
+   * Used by test {@link #passingTestShouldPassOneTimeWhenRepeatIsUnused()}
+   */
+  public static class PassingTestShouldPassOneTimeWhenRepeatIsUnused {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #zeroValueShouldThrowIllegalArgumentException()}
+   */
+  public static class ZeroValueShouldThrowIllegalArgumentException {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(0)
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #negativeValueShouldThrowIllegalArgumentException()}
+   */
+  public static class NegativeValueShouldThrowIllegalArgumentException {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(-1)
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #passingTestShouldBeSkippedWhenRepeatIsZero()}
+   */
+  public static class PassingTestShouldBeSkippedWhenRepeatIsZero {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(0)
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failingTestShouldFailOneTimeWhenRepeatIsOne()}
+   */
+  public static class FailingTestShouldFailOneTimeWhenRepeatIsOne {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(1)
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+
+  /**
+   * Used by test {@link #passingTestShouldPassOneTimeWhenRepeatIsOne()}
+   */
+  public static class PassingTestShouldPassOneTimeWhenRepeatIsOne {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(1)
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failingTestShouldFailOneTimeWhenRepeatIsTwo()}
+   */
+  public static class FailingTestShouldFailOneTimeWhenRepeatIsTwo {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(2)
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+
+  /**
+   * Used by test {@link #passingTestShouldPassTwoTimesWhenRepeatIsTwo()}
+   */
+  public static class PassingTestShouldPassTwoTimesWhenRepeatIsTwo {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(2)
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failingTestShouldFailOneTimeWhenRepeatIsThree()}
+   */
+  public static class FailingTestShouldFailOneTimeWhenRepeatIsThree {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(3)
+    public void doTest() throws Exception {
+      count++;
+      fail(ASSERTION_ERROR_MESSAGE);
+    }
+  }
+
+  /**
+   * Used by test {@link #passingTestShouldPassThreeTimesWhenRepeatIsThree()}
+   */
+  public static class PassingTestShouldPassThreeTimesWhenRepeatIsThree {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RepeatRule repeat = new RepeatRule();
+
+    @Test
+    @Repeat(3)
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithErrorTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithErrorTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithErrorTest.java
new file mode 100755
index 0000000..0e54298
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithErrorTest.java
@@ -0,0 +1,326 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.junit.Assert.fail;
+
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.Retry;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link RetryRule} involving global scope (ie rule affects all
+ * tests in the test class) with failures due to an {@code Error}.
+ * 
+ * @see com.gemstone.gemfire.test.junit.rules.RetryRule
+ */
+@Category(UnitTest.class)
+public class RetryRuleGlobalWithErrorTest {
+  
+  @Test
+  public void zeroIsIllegal() {
+    Result result = TestRunner.runTest(ZeroIsIllegal.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage(ZeroIsIllegal.message);
+    assertThat(ZeroIsIllegal.count).isEqualTo(0);
+  }
+  
+  @Test
+  public void failsWithOne() {
+    Result result = TestRunner.runTest(FailsWithOne.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsWithOne.message);
+    assertThat(FailsWithOne.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void passesWithOne() {
+    Result result = TestRunner.runTest(PassesWithOne.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesWithOne.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void passesWithUnused() {
+    Result result = TestRunner.runTest(PassesWhenUnused.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesWhenUnused.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void failsOnSecondAttempt() {
+    Result result = TestRunner.runTest(FailsOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnSecondAttempt.message);
+    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
+  }
+
+  @Test
+  public void passesOnSecondAttempt() {
+    Result result = TestRunner.runTest(PassesOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
+  }
+  
+  @Test
+  public void failsOnThirdAttempt() {
+    Result result = TestRunner.runTest(FailsOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnThirdAttempt.message);
+    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  @Test
+  public void passesOnThirdAttempt() {
+    Result result = TestRunner.runTest(PassesOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  /**
+   * Used by test {@link #zeroIsIllegal()}
+   */
+  public static class ZeroIsIllegal {
+
+    static final String message = "Retry count must be greater than zero";
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(0);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWithOne()}
+   */
+  public static class FailsWithOne {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(1);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      fail(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesWithOne()}
+   */
+  public static class PassesWithOne {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(1);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #passesWithUnused()}
+   */
+  public static class PassesWhenUnused {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(2);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnSecondAttempt()}
+   */
+  public static class FailsOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(2);
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      fail(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnSecondAttempt()}
+   */
+  public static class PassesOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(2);
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      if (count < 2) {
+        message = "Failing " + count;
+        fail(message);
+      }
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnThirdAttempt()}
+   */
+  public static class FailsOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(3);
+
+    @Test
+    @Retry(3)
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      fail(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnThirdAttempt()}
+   */
+  public static class PassesOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(3);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      if (count < 3) {
+        message = "Failing " + count;
+        fail(message);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithExceptionTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithExceptionTest.java
new file mode 100755
index 0000000..8ece92b
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleGlobalWithExceptionTest.java
@@ -0,0 +1,332 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.Retry;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link RetryRule} involving global scope (ie rule affects all
+ * tests in the test class) with failures due to an {@code Exception}.
+ * 
+ * @see com.gemstone.gemfire.test.junit.rules.RetryRule
+ */
+@Category(UnitTest.class)
+public class RetryRuleGlobalWithExceptionTest {
+  
+  @Test
+  public void zeroIsIllegal() {
+    Result result = TestRunner.runTest(ZeroIsIllegal.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(IllegalArgumentException.class).hasMessage(ZeroIsIllegal.message);
+    assertThat(ZeroIsIllegal.count).isEqualTo(0);
+  }
+  
+  @Test
+  public void failsWithOne() {
+    Result result = TestRunner.runTest(FailsWithOne.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsWithOne.message);
+    assertThat(FailsWithOne.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void passesWithOne() {
+    Result result = TestRunner.runTest(PassesWithOne.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+  }
+  
+  @Test
+  public void passesWithUnused() {
+    Result result = TestRunner.runTest(PassesWhenUnused.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+  }
+  
+  @Test
+  public void failsOnSecondAttempt() {
+    Result result = TestRunner.runTest(FailsOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnSecondAttempt.message);
+    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
+  }
+
+  @Test
+  public void passesOnSecondAttempt() {
+    Result result = TestRunner.runTest(PassesOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
+  }
+  
+  @Test
+  public void failsOnThirdAttempt() {
+    Result result = TestRunner.runTest(FailsOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnThirdAttempt.message);
+    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  @Test
+  public void passesOnThirdAttempt() {
+    Result result = TestRunner.runTest(PassesOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  /**
+   * Custom exception used by several tests
+   */
+  public static class CustomException extends Exception {
+    public CustomException(final String message) {
+      super(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #zeroIsIllegal()}
+   */
+  public static class ZeroIsIllegal {
+
+    static int count = 0;
+    static final String message = "Retry count must be greater than zero";
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(0);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failsWithOne()}
+   */
+  public static class FailsWithOne {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(1);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      throw new CustomException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesWithOne()}
+   */
+  public static class PassesWithOne {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(1);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #passesWithUnused()}
+   */
+  public static class PassesWhenUnused {
+
+    static int count = 0;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(2);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnSecondAttempt()}
+   */
+  public static class FailsOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(2);
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      throw new CustomException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnSecondAttempt()}
+   */
+  public static class PassesOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+    
+    @Rule
+    public RetryRule retryRule = new RetryRule(2);
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      if (count < 2) {
+        message = "Failing " + count;
+        throw new CustomException(message);
+      }
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnThirdAttempt()}
+   */
+  public static class FailsOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(3);
+
+    @Test
+    @Retry(3)
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      throw new CustomException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnThirdAttempt()}
+   */
+  public static class PassesOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule(3);
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      if (count < 3) {
+        message = "Failing " + count;
+        throw new CustomException(message);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithErrorTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithErrorTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithErrorTest.java
new file mode 100755
index 0000000..3f0642c
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithErrorTest.java
@@ -0,0 +1,265 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.junit.Assert.fail;
+
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.Retry;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link RetryRule} involving local scope (ie rule affects
+ * only the test methods annotated with {@code @Retry}) with failures due to
+ * an {@code Error}.
+ */
+@Category(UnitTest.class)
+public class RetryRuleLocalWithErrorTest {
+
+  @Test
+  public void failsUnused() {
+    Result result = TestRunner.runTest(FailsUnused.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsUnused.message);
+    assertThat(FailsUnused.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void passesUnused() {
+    Result result = TestRunner.runTest(PassesUnused.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesUnused.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void failsOnSecondAttempt() {
+    Result result = TestRunner.runTest(FailsOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnSecondAttempt.message);
+    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
+  }
+
+  @Test
+  public void passesOnSecondAttempt() {
+    Result result = TestRunner.runTest(PassesOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
+  }
+  
+  @Test
+  public void failsOnThirdAttempt() {
+    Result result = TestRunner.runTest(FailsOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(AssertionError.class).hasMessage(FailsOnThirdAttempt.message);
+    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  @Test
+  public void passesOnThirdAttempt() {
+    Result result = TestRunner.runTest(PassesOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  /**
+   * Used by test {@link #failsUnused()}
+   */
+  public static class FailsUnused {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      fail(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesUnused()}
+   */
+  public static class PassesUnused {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnSecondAttempt()}
+   */
+  public static class FailsOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      fail(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnSecondAttempt()}
+   */
+  public static class PassesOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      if (count < 2) {
+        message = "Failing " + count;
+        fail(message);
+      }
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnThirdAttempt()}
+   */
+  public static class FailsOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(3)
+    public void doTest() throws Exception {
+      count++;
+
+      message = "Failing " + count;
+      fail(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnThirdAttempt()}
+   */
+  public static class PassesOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(3)
+    public void doTest() throws Exception {
+      count++;
+
+      if (count < 3) {
+        message = "Failing " + count;
+        fail(message);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithExceptionTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithExceptionTest.java
new file mode 100755
index 0000000..6322e88
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RetryRuleLocalWithExceptionTest.java
@@ -0,0 +1,276 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+import com.gemstone.gemfire.test.junit.Retry;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link RetryRule} involving local scope (ie rule affects
+ * only the test methods annotated with {@code @Retry}) with failures due to
+ * an {@code Exception}.
+ *
+ * @see com.gemstone.gemfire.test.junit.Retry
+ * @see com.gemstone.gemfire.test.junit.rules.RetryRule
+ */
+@Category(UnitTest.class)
+public class RetryRuleLocalWithExceptionTest {
+
+  @Test
+  public void failsUnused() {
+    Result result = TestRunner.runTest(FailsUnused.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsUnused.message);
+    assertThat(FailsUnused.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void passesUnused() {
+    Result result = TestRunner.runTest(PassesUnused.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesUnused.count).isEqualTo(1);
+  }
+  
+  @Test
+  public void failsOnSecondAttempt() {
+    Result result = TestRunner.runTest(FailsOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnSecondAttempt.message);
+    assertThat(FailsOnSecondAttempt.count).isEqualTo(2);
+  }
+
+  @Test
+  public void passesOnSecondAttempt() {
+    Result result = TestRunner.runTest(PassesOnSecondAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnSecondAttempt.count).isEqualTo(2);
+  }
+  
+  @Test
+  public void failsOnThirdAttempt() {
+    Result result = TestRunner.runTest(FailsOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isFalse();
+    
+    List<Failure> failures = result.getFailures();
+    assertThat(failures.size()).as("Failures: " + failures).isEqualTo(1);
+
+    Failure failure = failures.get(0);
+    assertThat(failure.getException()).isExactlyInstanceOf(CustomException.class).hasMessage(FailsOnThirdAttempt.message);
+    assertThat(FailsOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  @Test
+  public void passesOnThirdAttempt() {
+    Result result = TestRunner.runTest(PassesOnThirdAttempt.class);
+    
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(PassesOnThirdAttempt.count).isEqualTo(3);
+  }
+
+  /**
+   * Custom exception used by several tests
+   */
+  public static class CustomException extends Exception {
+    public CustomException(final String message) {
+      super(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #failsUnused()}
+   */
+  public static class FailsUnused {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      throw new CustomException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesUnused()}
+   */
+  public static class PassesUnused {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    public void doTest() throws Exception {
+      count++;
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnSecondAttempt()}
+   */
+  public static class FailsOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      message = "Failing " + count;
+      throw new CustomException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnSecondAttempt()}
+   */
+  public static class PassesOnSecondAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(2)
+    public void doTest() throws Exception {
+      count++;
+      if (count < 2) {
+        message = "Failing " + count;
+        throw new CustomException(message);
+      }
+    }
+  }
+
+  /**
+   * Used by test {@link #failsOnThirdAttempt()}
+   */
+  public static class FailsOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(3)
+    public void doTest() throws Exception {
+      count++;
+
+      message = "Failing " + count;
+      throw new CustomException(message);
+    }
+  }
+
+  /**
+   * Used by test {@link #passesOnThirdAttempt()}
+   */
+  public static class PassesOnThirdAttempt {
+
+    static int count = 0;
+    static String message = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+      count = 0;
+      message = null;
+    }
+    
+    @Rule
+    public RetryRule retryRule = new RetryRule();
+
+    @Test
+    @Retry(3)
+    public void doTest() throws Exception {
+      count++;
+
+      if (count < 3) {
+        message = "Failing " + count;
+        throw new CustomException(message);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
new file mode 100755
index 0000000..ce98dfe
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/RuleListTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.junit.rules;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExternalResource;
+import org.junit.runner.Result;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Unit tests for {@link RuleList}.
+ */
+public class RuleListTest {
+
+  private static AtomicInteger counter;
+  private static Invocations[] invocations;
+
+  @BeforeClass
+  public static void setUpClass() {
+    counter = new AtomicInteger();
+    invocations = new Invocations[] { new Invocations(counter), new Invocations(counter), new Invocations(counter) };
+  }
+
+  @AfterClass
+  public static void tearDownClass() {
+    counter = null;
+    invocations = null;
+    ThreeRules.ruleListStatic = null;
+  }
+
+  @Test
+  public void firstShouldBeFirstBeforeLastAfter() {
+    Result result = TestRunner.runTest(ThreeRules.class);
+
+    assertThat(result.wasSuccessful()).isTrue();
+
+    assertThat(counter.get()).isEqualTo(9);
+
+    assertThat(invocations[0].beforeInvocation).isEqualTo(1);
+    assertThat(invocations[1].beforeInvocation).isEqualTo(2);
+    assertThat(invocations[2].beforeInvocation).isEqualTo(3);
+
+    assertThat(invocations[0].testInvocation).isEqualTo(4);
+    assertThat(invocations[1].testInvocation).isEqualTo(5);
+    assertThat(invocations[2].testInvocation).isEqualTo(6);
+
+    assertThat(invocations[2].afterInvocation).isEqualTo(7);
+    assertThat(invocations[1].afterInvocation).isEqualTo(8);
+    assertThat(invocations[0].afterInvocation).isEqualTo(9);
+  }
+
+  /**
+   * Used by test {@link #firstShouldBeFirstBeforeLastAfter()}
+   */
+  public static class ThreeRules {
+
+    static RuleList ruleListStatic;
+
+    public SpyRule ruleOne = new SpyRule("ruleOne", invocations[0]);
+    public SpyRule ruleTwo = new SpyRule("ruleTwo", invocations[1]);
+    public SpyRule ruleThree = new SpyRule("ruleThree", invocations[2]);
+
+    @Rule
+    public RuleList ruleList = new RuleList().add(ruleThree).add(ruleTwo).add(ruleOne);
+
+    @Test
+    public void doTest() throws Exception {
+      ruleListStatic = ruleList;
+      invocations[0].invokedTest();
+      invocations[1].invokedTest();
+      invocations[2].invokedTest();
+    }
+  }
+
+  /**
+   * Structure of rule callback and test invocations
+   */
+  public static class Invocations {
+
+    private final AtomicInteger counter;
+    int beforeInvocation = 0;
+    int testInvocation = 0;
+    int afterInvocation = 0;
+
+    Invocations(AtomicInteger counter) {
+      this.counter = counter;
+    }
+
+    void invokedTest() {
+      testInvocation = counter.incrementAndGet();
+    }
+    void invokedBefore() {
+      beforeInvocation = counter.incrementAndGet();
+    }
+    void invokedAfter() {
+      afterInvocation = counter.incrementAndGet();
+    }
+
+    @Override
+    public String toString() {
+      return "Invocations{" + "counter=" + counter + ", beforeInvocation=" + beforeInvocation + ", testInvocation=" + testInvocation + ", afterInvocation=" + afterInvocation + '}';
+    }
+  }
+
+  /**
+   * Implementation of TestRule that records the order of callbacks invoked on
+   * it. Used by {@link RuleListTest}.
+   */
+  public static class SpyRule extends ExternalResource {
+
+    static SpyRuleBuilder builder() {
+      return new SpyRuleBuilder();
+    }
+
+    private final String name;
+    private final Invocations invocations;
+    private final Throwable beforeThrowable;
+
+    SpyRule(String name, Invocations invocations) {
+      this.name = name;
+      this.invocations = invocations;
+      this.beforeThrowable = null;
+    }
+
+    SpyRule(SpyRuleBuilder builder) {
+      this.name = builder.name;
+      this.invocations = builder.invocations;
+      this.beforeThrowable = builder.beforeThrowable;
+    }
+
+    Invocations invocations() {
+      return this.invocations;
+    }
+
+    void test() {
+      this.invocations.invokedTest();
+    }
+
+    @Override
+    protected void before() throws Throwable {
+      this.invocations.invokedBefore();
+      if (this.beforeThrowable != null) {
+        throw this.beforeThrowable;
+      }
+    }
+
+    @Override
+    protected void after() {
+      this.invocations.invokedAfter();
+    }
+
+    @Override
+    public String toString() {
+      return "SpyRule{" + "name='" + name + '\'' + '}';
+    }
+  }
+
+  /**
+   * Builder for more control of constructing an instance of {@link SpyRule}
+   */
+  public static class SpyRuleBuilder {
+
+    String name;
+    Invocations invocations;
+    Throwable beforeThrowable;
+
+    SpyRuleBuilder withName(String name) {
+      this.name = name;
+      return this;
+    }
+
+    SpyRuleBuilder withInvocations(Invocations invocations) {
+      this.invocations = invocations;
+      return this;
+    }
+
+    SpyRuleBuilder beforeThrows(Throwable throwable) {
+      this.beforeThrowable = throwable;
+      return this;
+    }
+
+    SpyRule build() {
+      return new SpyRule(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/TestRunner.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/TestRunner.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/TestRunner.java
new file mode 100755
index 0000000..86addb5
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/TestRunner.java
@@ -0,0 +1,35 @@
+/*
+ * 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.junit.rules;
+
+import org.junit.runner.JUnitCore;
+import org.junit.runner.Request;
+import org.junit.runner.Result;
+
+/**
+ * Used by JUnit rule unit tests to execute inner test cases.
+ */
+public class TestRunner {
+
+  protected TestRunner() {
+  }
+
+  public static Result runTest(Class<?> test) {
+    JUnitCore junitCore = new JUnitCore();
+    return junitCore.run(Request.aClass(test).getRunner());
+  }
+}



[2/7] incubator-geode git commit: GEODE-953: Cleanup geode-junit

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RuleAndClassRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RuleAndClassRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RuleAndClassRuleTest.java
deleted file mode 100755
index 9c050e9..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/RuleAndClassRuleTest.java
+++ /dev/null
@@ -1,138 +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.junit.rules.tests;
-
-import static com.gemstone.gemfire.test.junit.rules.tests.TestRunner.*;
-import static org.assertj.core.api.Assertions.*;
-
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestRule;
-import org.junit.runner.Description;
-import org.junit.runner.Result;
-import org.junit.runners.model.Statement;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class RuleAndClassRuleTest {
-
-  @Test
-  public void usingRuleAsRuleAndClassRuleShouldInvokeBeforeClass() {
-    Result result = runTest(UsingRuleAsRuleAndClassRule.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(UsingRuleAsRuleAndClassRule.staticRule.beforeClassInvoked).isEqualTo(true);
-  }
-  
-  @Test
-  public void usingRuleAsRuleAndClassRuleShouldInvokeAfterClass() {
-    Result result = runTest(UsingRuleAsRuleAndClassRule.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(UsingRuleAsRuleAndClassRule.staticRule.afterClassInvoked).isEqualTo(true);
-  }
-
-  @Test
-  public void usingRuleAsRuleAndClassRuleShouldInvokeBefore() {
-    Result result = runTest(UsingRuleAsRuleAndClassRule.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(UsingRuleAsRuleAndClassRule.staticRule.beforeInvoked).isEqualTo(true);
-  }
-
-  @Test
-  public void usingRuleAsRuleAndClassRuleShouldInvokeAfter() {
-    Result result = runTest(UsingRuleAsRuleAndClassRule.class);
-    
-    assertThat(result.wasSuccessful()).isTrue();
-    assertThat(UsingRuleAsRuleAndClassRule.staticRule.afterInvoked).isEqualTo(true);
-  }
-
-  public static class SpyRule implements TestRule {
-    boolean beforeClassInvoked;
-    boolean afterClassInvoked;
-    boolean beforeInvoked;
-    boolean afterInvoked;
-    
-    @Override
-    public Statement apply(final Statement base, final Description description) {
-      if (description.isTest()) {
-        return statement(base);
-      } else if (description.isSuite()) {
-        return statementClass(base);
-      }
-      return base;
-    }
-
-    private Statement statement(final Statement base) {
-      return new Statement() {
-        @Override
-        public void evaluate() throws Throwable {
-          before();
-          try {
-            base.evaluate();
-          } finally {
-            after();
-          }
-        }
-      };
-    }
-    
-    private Statement statementClass(final Statement base) {
-      return new Statement() {
-        @Override
-        public void evaluate() throws Throwable {
-          beforeClass();
-          try {
-            base.evaluate();
-          } finally {
-            afterClass();
-          }
-        }
-      };
-    }
-    
-    private void beforeClass() {
-      this.beforeClassInvoked = true;
-    }
-    
-    private void afterClass() {
-      this.afterClassInvoked = true;
-    }
-    
-    private void before() {
-      this.beforeInvoked = true;
-    }
-    
-    private void after() {
-      this.afterInvoked = true;
-    }
-  };
-  
-  public static class UsingRuleAsRuleAndClassRule {
-    @ClassRule
-    public static SpyRule staticRule = new SpyRule();
-    @Rule
-    public SpyRule rule = staticRule;
-    @Test
-    public void doTest() throws Exception {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5342935d/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/TestRunner.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/TestRunner.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/TestRunner.java
deleted file mode 100755
index 684f452..0000000
--- a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/tests/TestRunner.java
+++ /dev/null
@@ -1,37 +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.junit.rules.tests;
-
-import org.junit.runner.JUnitCore;
-import org.junit.runner.Request;
-import org.junit.runner.Result;
-
-/**
- * Used by Rule Unit Tests to execute Test Cases.
- * 
- * @author Kirk Lund
- */
-public class TestRunner {
-
-  protected TestRunner() {
-  }
-  
-  public static Result runTest(Class<?> test) {
-    JUnitCore junitCore = new JUnitCore();
-    return junitCore.run(Request.aClass(test).getRunner());
-  }
-}