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/01/21 20:08:51 UTC

[05/19] incubator-geode git commit: GEODE-601: Fix intermittent exception in DiskRegionJUnitTest.

GEODE-601: Fix intermittent exception in DiskRegionJUnitTest.

An expected DiskAccessException causes the cache to close
asynchronously. That leads to a race condition during test tearDown,
between checking that the cache is closed and it's access. Now wait
for the cache to close before tearDown.
Refactors:
 Moved putSuccessful and exceptionOccurred inside Puts
 Remove unnecessary thread creation for Puts in DiskRegionJUnitTest
 Expose _testHandleDiskAccessException through method

This closes #72


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

Branch: refs/heads/feature/GEODE-715
Commit: c465541d919652d3d4a1305ea6ddcd7c31513188
Parents: f45f3a9
Author: Shreedhar Hardikar <sh...@pivotal.io>
Authored: Thu Jan 7 14:01:21 2016 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Tue Jan 19 15:01:44 2016 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/DiskStoreImpl.java   |   4 +-
 .../internal/cache/Bug39079DUnitTest.java       |   4 +-
 .../internal/cache/DiskRegionJUnitTest.java     | 172 +++++++------------
 .../internal/cache/DiskRegionTestingBase.java   |   1 -
 4 files changed, 66 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c465541d/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
index 99524d9..e4ef21d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskStoreImpl.java
@@ -368,7 +368,7 @@ public class DiskStoreImpl implements DiskStore {
    */
   private DiskStoreID diskStoreID;
 
-  final CountDownLatch _testHandleDiskAccessException = new CountDownLatch(1);
+  private final CountDownLatch _testHandleDiskAccessException = new CountDownLatch(1);
   
   private final ThreadPoolExecutor diskStoreTaskPool;
   
@@ -2311,7 +2311,7 @@ public class DiskStoreImpl implements DiskStore {
     close(false);
   }
 
-  void waitForClose() {
+  protected void waitForClose() {
     if (diskException.get() != null) {
       try {
         _testHandleDiskAccessException.await();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c465541d/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
index f17d896..4927abe 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
@@ -356,7 +356,7 @@ public class Bug39079DUnitTest extends CacheTestCase {
     assertNotNull(gemfirecache);
   }
   
-  private static void validateRuningBridgeServerList() throws Exception{
+  private static void validateRuningBridgeServerList(){
     /*Region region = gemfirecache.getRegion(Region.SEPARATOR + REGION_NAME);
     assertNotNull(region);*/
     try {        
@@ -374,7 +374,7 @@ public class Bug39079DUnitTest extends CacheTestCase {
         fail("test failed due to ", e);
       }
       
-      ((LocalRegion)region).getDiskRegion().getDiskStore()._testHandleDiskAccessException.await();
+      ((LocalRegion) region).getDiskStore().waitForClose();
       assertTrue(region.getRegionService().isClosed());
       
       region = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c465541d/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
index cb9d3ff..e3b5ab0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
@@ -548,7 +548,6 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
    * 
    * @author ashahid
    */
-  protected volatile  boolean[] putSuccessfull = new boolean[3];
 
   @Test
   public void testSingleDirectoryNotHanging()
@@ -568,31 +567,21 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
         diskRegionProperties, Scope.LOCAL);
 
     Puts puts = new Puts(region);
+    puts.performPuts();
 
-    Thread thread1 = new Thread(puts);
-    thread1.setDaemon(true);
-    thread1.start();
-
-    long startTime = System.currentTimeMillis();
-    DistributedTestCase.join(thread1, 40 * 1000, null);
-    long interval = System.currentTimeMillis() - startTime;
-    if (interval > 40000) {
-      fail(" Test took too long in going to join, it should have exited before 40000 ms");
-    }
-
-    if (!putSuccessfull[0]) {
+    if (!puts.putSuccessful(0)) {
       fail(" first put did not succeed");
     }
 
-    if (!putSuccessfull[1]) {
+    if (!puts.putSuccessful(1)) {
       fail(" second put did not succeed");
     }
 
-    if (!putSuccessfull[2]) {
+    if (!puts.putSuccessful(2)) {
       fail(" third put did not succeed");
     }
 
-    if (exceptionOccured) {
+    if (puts.exceptionOccurred()) {
       fail(" Exception was not supposed to occur but did occur");
     }
     closeDown();
@@ -612,42 +601,21 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
         diskRegionProperties, Scope.LOCAL);
 
     Puts puts = new Puts(region);
-    Thread thread1 = new Thread(puts);
-    thread1.setDaemon(true);
-    thread1.start();
-    if (!putsHaveStarted) {
-      synchronized (puts) {
-        if (!putsHaveStarted) {
-          try {
-            puts.wait();
-          }
-          catch (InterruptedException e) {
-            fail("Unexpected interrupted exception");
-          }
-        }
-      }
-    }
-   
-    long startTime = System.currentTimeMillis();
-    DistributedTestCase.join(thread1, 9 * 1000, null);
-    long interval = System.currentTimeMillis() - startTime;
-    if (interval > 9000) {
-      fail(" Test took too long in going to join, it should have exited before 9000 ms");
-    }
+    puts.performPuts();
 
-    if (!putSuccessfull[0]) {
+    if (!puts.putSuccessful(0)) {
       fail(" first put did not succeed");
     }
 
-    if (!putSuccessfull[1]) {
+    if (!puts.putSuccessful(1)) {
       fail(" second put did not succeed");
     }
 
-    if (!putSuccessfull[2]) {
+    if (!puts.putSuccessful(2)) {
       fail(" third put did not succeed");
     }
 
-    if (exceptionOccured) {
+    if (puts.exceptionOccurred()) {
       fail(" Exception was not supposed to occur but did occur");
     }
 
@@ -675,46 +643,25 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
     }
 
     Puts puts = new Puts(region, 1026);
+    puts.performPuts();
 
-    Thread thread1 = new Thread(puts);
-    thread1.setDaemon(true);
-    thread1.start();
-
-    
-    if (!putsHaveStarted) {
-      synchronized (puts) {
-        if (!putsHaveStarted) {
-          try {
-            puts.wait();
-          }
-          catch (InterruptedException e) {
-            fail("Unexpected interrupted exception");
-          }
-        }
-      }
-    }
-    long startTime = System.currentTimeMillis();
-    DistributedTestCase.join(thread1, 33 * 1000, null);
-    long interval = System.currentTimeMillis() - startTime;
-    if (interval > 33000) {
-      fail(" Test took too long in going to join, it should have exited before 33000 ms");
-    }
-
-    if (!exceptionOccured) {
+    if (!puts.exceptionOccurred()) {
       fail(" Exception was supposed to occur but did not occur");
     }
-    if (putSuccessfull[0]) {
+    if (puts.putSuccessful(0)) {
       fail(" first put did succeed when it was not supposed to");
     }
 
-    if (putSuccessfull[1]) {
+    if (puts.putSuccessful(1)) {
       fail(" second put did  succeed  when it was not supposed to");
     }
 
-    if (putSuccessfull[2]) {
+    if (puts.putSuccessful(2)) {
       fail(" third put did  succeed  when it was not supposed to");
     }
 
+    // if the exception occurred then the region should be closed already
+    ((LocalRegion) region).getDiskStore().waitForClose();
   }
 
   /**
@@ -788,6 +735,8 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
 
     private int dataSize = 1024;
     private Region region;
+    private volatile boolean[] putSuccessful = new boolean[3];
+    private volatile boolean exceptionOccurred = false;
 
     Puts(Region region) {
       this.region = region;
@@ -796,15 +745,24 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       this.region = region;
       this.dataSize = dataSize;
     }
-      
 
-    public void run()
-    {
-     
-      exceptionOccured = false;
-      putSuccessfull[0] = false;
-      putSuccessfull[1] = false;
-      putSuccessfull[2] = false;
+    public boolean exceptionOccurred() {
+      return exceptionOccurred;
+    }
+
+    public boolean putSuccessful(int index){
+      return putSuccessful[index];
+    }
+
+    public void run() {
+      performPuts();
+    }
+ 
+    public void performPuts(){
+      exceptionOccurred = false;
+      putSuccessful[0] = false;
+      putSuccessful[1] = false;
+      putSuccessful[2] = false;
 
       try {
         byte[] bytes = new byte[this.dataSize];
@@ -813,14 +771,14 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
           this.notify();
         }
         region.put("1", bytes);
-        putSuccessfull[0] = true;
+        putSuccessful[0] = true;
         region.put("2", bytes);
-        putSuccessfull[1] = true;
+        putSuccessful[1] = true;
         region.put("3", bytes);
-        putSuccessfull[2] = true;
+        putSuccessful[2] = true;
       }
       catch (DiskAccessException e) {
-        exceptionOccured = true;
+        exceptionOccurred = true;
       }
     }
   }
@@ -835,8 +793,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
     File dir = new File("testSingleDirectoryNotHanging");
     dir.mkdir();
     dir.deleteOnExit();
-    File[] dirs = new File[1];
-    dirs[0] = dir;
+    File[] dirs = new File[] {dir};
     int[] dirSizes = { 2048 };
     diskRegionProperties.setDiskDirsAndSizes(dirs, dirSizes);
     diskRegionProperties.setMaxOplogSize(2097152);
@@ -845,33 +802,30 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
         diskRegionProperties, Scope.LOCAL);
 
     Puts puts = new Puts(region);
+    puts.performPuts();
 
-    Thread thread1 = new Thread(puts);
-    thread1.setDaemon(true);
-    thread1.start();
-    DistributedTestCase.join(thread1, 30 * 1000, null);
-
-    if (!putSuccessfull[0]) {
+    if (!puts.putSuccessful(0)) {
       fail(" first put did not succeed");
     }
 
-    if (putSuccessfull[1]) {
+    if (puts.putSuccessful(1)) {
       fail(" second put should not succeed");
     }
 
-    if (!exceptionOccured) {
+    if (!puts.exceptionOccurred()) {
       fail(" Exception was supposed to occur but did not occur");
     }
-    closeDown();
+    // if the exception occurred then the region should be closed already
+    ((LocalRegion) region).getDiskStore().waitForClose();
 
+    closeDown();
   }
 
   /**
    * DiskRegDiskAccessExceptionTest : Disk region test for DiskAccessException.
-   * @throws InterruptedException 
    */
   @Test
-  public void testDiskFullExcep() throws InterruptedException
+  public void testDiskFullExcep()
   {
     int[] diskDirSize1 = new int[4];
     diskDirSize1[0] = (2048 + 500);
@@ -921,9 +875,8 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
                      + "</ExpectedException>");
     }
 
-    // if the exception Occurred then the region should be closed already
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+    // if the exception occurred then the region should be closed already
+    ((LocalRegion) region).getDiskStore().waitForClose();
     assertEquals(true, cache.isClosed());
   }
 
@@ -981,10 +934,9 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
   }
   /**
    * DiskRegDiskAccessExceptionTest : Disk region test for DiskAccessException.
-   * @throws InterruptedException 
    */
   @Test
-  public void testDiskFullExcepOverflowOnly() throws InterruptedException
+  public void testDiskFullExcepOverflowOnly()
   {
     int[] diskDirSize1 = new int[4];
     diskDirSize1[0] = (2048 + 500);
@@ -1038,8 +990,8 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
                      + "</ExpectedException>");
     }
 
-    // if the exception Occurred then the region should be closed already
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+    // if the exception occurred then the region should be closed already
+    ((LocalRegion) region).getDiskStore().waitForClose();
     assertEquals(true, cache.isClosed());
   }
   
@@ -1983,7 +1935,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       // OK
     }
 
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+    ((LocalRegion) region).getDiskStore().waitForClose();
     assertTrue(cache.isClosed());
     region = null;
   }
@@ -2047,7 +1999,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       // OK
     }
 
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+    ((LocalRegion) region).getDiskStore().waitForClose();
     assertTrue(cache.isClosed());
     region = null;
   }
@@ -2113,7 +2065,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
     catch (DiskAccessException dae) {
       // OK
     }
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+    ((LocalRegion) region).getDiskStore().waitForClose();
     assertTrue(cache.isClosed());
     region = null;
   }
@@ -2178,7 +2130,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       // OK
     }
 
-    ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+    ((LocalRegion) region).getDiskStore().waitForClose();
     assertTrue(cache.isClosed());
     region = null;
   }
@@ -2212,7 +2164,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       catch (DiskAccessException dae) {
         //OK
       }        
-      ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+      ((LocalRegion) region).getDiskStore().waitForClose();
       assertTrue(cache.isClosed());
       region = null;
   }
@@ -2245,7 +2197,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       catch (DiskAccessException dae) {
         //OK
       }       
-      ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+      ((LocalRegion) region).getDiskStore().waitForClose();
       assertTrue(cache.isClosed());
       region = null;
   }
@@ -2326,7 +2278,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
         }                
       }, 5000, 500, true);
       
-      ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+      ((LocalRegion) region).getDiskStore().waitForClose();
       assertTrue(cache.isClosed());
       region = null;
   }
@@ -2364,7 +2316,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       }catch(DiskAccessException dae) {
         //OK expected
       }
-      ((LocalRegion) region).getDiskStore()._testHandleDiskAccessException.await();
+      ((LocalRegion) region).getDiskStore().waitForClose();
       assertTrue(cache.isClosed());        
       region = null;
       List bsRunning = cache.getCacheServers();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c465541d/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionTestingBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionTestingBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionTestingBase.java
index 8a425ee..d0911b8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionTestingBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionTestingBase.java
@@ -134,7 +134,6 @@ public class DiskRegionTestingBase
       if (cache != null && !cache.isClosed()) {
         for (Iterator itr = cache.rootRegions().iterator(); itr.hasNext();) {
           Region root = (Region)itr.next();
-//          String name = root.name.getMethodName();
           if(root.isDestroyed() || root instanceof HARegion) {
             continue;
           }