You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by km...@apache.org on 2016/12/23 21:10:12 UTC

[19/25] geode git commit: GEODE-2109: Replacing ExecutorService.submit calls with ExecutorService.execute call.

GEODE-2109: Replacing ExecutorService.submit calls with ExecutorService.execute call.

This will allow exceptions from those threads to be logged.

Refactored DiskStore task for delayed writes, we cannot replace
expensive writes tasks submit call with execute as later we check if
write call is completed or not.  Replaced submit call by execute in case
of DiskStore tasks like compactions, creating KRF's.

Replaced submit call for GIITask and RemoveMember tasks. GIITask is used
when adding member or starting managing activity when node becomes
managing node. While adding member we can make the
ExecutorService.execute call. Did not change the call for GIITask when
invoked from managing activity as possible exception is handled.

Added LoggingThreadGroup for SingleHopClientExecutor

This closes #296


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

Branch: refs/heads/feature/GEODE-2231
Commit: bb3d20f41d8b9f4e663d6ae7ddbd83ad8de01e77
Parents: 0b01117
Author: Deepak Dixit <de...@ampool.io>
Authored: Wed Nov 23 21:33:14 2016 +0530
Committer: Karen Miller <km...@pivotal.io>
Committed: Fri Dec 23 13:09:28 2016 -0800

----------------------------------------------------------------------
 .../internal/SingleHopClientExecutor.java       |  8 ++-
 .../FunctionExecutionPooledExecutor.java        |  2 +-
 .../distributed/internal/InternalLocator.java   |  4 +-
 .../geode/internal/cache/DiskStoreImpl.java     | 40 ++++++++++++-
 .../internal/cache/PRHARedundancyProvider.java  |  2 +-
 .../geode/internal/cache/lru/HeapEvictor.java   |  2 +-
 .../CustomEntryConcurrentHashMap.java           |  2 +-
 .../management/internal/FederatingManager.java  | 26 ++++++---
 ...ientExecutorSubmitTaskWithExceptionTest.java | 61 ++++++++++++++++++++
 .../management/MemoryThresholdsDUnitTest.java   |  6 ++
 .../internal/cache/EvictionStatsDUnitTest.java  | 25 ++++++++
 .../PartitionedRegionEvictionDUnitTest.java     |  7 +++
 12 files changed, 169 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/cache/client/internal/SingleHopClientExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/SingleHopClientExecutor.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/SingleHopClientExecutor.java
index 88a86cf..4d40acd 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/SingleHopClientExecutor.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/SingleHopClientExecutor.java
@@ -27,6 +27,8 @@ import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.geode.internal.logging.LoggingThreadGroup;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.GemFireException;
@@ -57,7 +59,9 @@ public class SingleHopClientExecutor {
     AtomicInteger threadNum = new AtomicInteger();
 
     public Thread newThread(final Runnable r) {
-      Thread result = new Thread(r, "Function Execution Thread-" + threadNum.incrementAndGet());
+      Thread result =
+          new Thread(LoggingThreadGroup.createThreadGroup("FunctionExecutionThreadGroup", logger),
+              r, "Function Execution Thread-" + threadNum.incrementAndGet());
       result.setDaemon(true);
       return result;
     }
@@ -384,7 +388,7 @@ public class SingleHopClientExecutor {
   }
 
   static void submitTask(Runnable task) {
-    execService.submit(task);
+    execService.execute(task);
   }
 
   // Find out what exception to throw?

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/distributed/internal/FunctionExecutionPooledExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/FunctionExecutionPooledExecutor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/FunctionExecutionPooledExecutor.java
index 34f1296..a519c0e 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/FunctionExecutionPooledExecutor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/FunctionExecutionPooledExecutor.java
@@ -131,7 +131,7 @@ public class FunctionExecutionPooledExecutor extends ThreadPoolExecutor {
               Runnable task = takeQueue.take();
               if (forFnExec) {
                 if (!putQueue.offer(task, retryFor, TimeUnit.MILLISECONDS)) {
-                  submit(task);
+                  execute(task);
                 }
               } else {
                 putQueue.put(task);

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 6bae8ba..59488ad 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -1158,7 +1158,7 @@ public class InternalLocator extends Locator implements ConnectListener {
       this.isSharedConfigurationStarted = true;
       if (isSharedConfigurationEnabled()) {
         ExecutorService es = newCache.getDistributionManager().getThreadPool();
-        es.submit(new SharedConfigurationRunnable());
+        es.execute(new SharedConfigurationRunnable());
       }
       if (!this.server.isAlive()) {
         logger.info("Locator restart: starting TcpServer");
@@ -1435,7 +1435,7 @@ public class InternalLocator extends Locator implements ConnectListener {
       this.isSharedConfigurationStarted = true;
       installSharedConfigStatus();
       ExecutorService es = gfc.getDistributionManager().getThreadPool();
-      es.submit(new SharedConfigurationRunnable());
+      es.execute(new SharedConfigurationRunnable());
     } else {
       logger.info("Cluster configuration service is disabled");
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
index f73e3f9..5affdb5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
@@ -4559,7 +4559,7 @@ public class DiskStoreImpl implements DiskStore {
    * tasks may take a while.
    */
   public boolean executeDiskStoreTask(final Runnable runnable) {
-    return executeDiskStoreTask(runnable, this.diskStoreTaskPool) != null;
+    return executeDiskStoreAsyncTask(runnable, this.diskStoreTaskPool);
   }
 
   /**
@@ -4617,6 +4617,32 @@ public class DiskStoreImpl implements DiskStore {
     return result;
   }
 
+  private boolean executeDiskStoreAsyncTask(final Runnable runnable, ThreadPoolExecutor executor) {
+    // schedule another thread to do it
+    incBackgroundTasks();
+    boolean isTaskAccepted = executeDiskStoreAsyncTask(new DiskStoreTask() {
+      public void run() {
+        try {
+          markBackgroundTaskThread(); // for bug 42775
+          // getCache().getCachePerfStats().decDiskTasksWaiting();
+          runnable.run();
+        } finally {
+          decBackgroundTasks();
+        }
+      }
+
+      public void taskCancelled() {
+        decBackgroundTasks();
+      }
+    }, executor);
+
+    if (!isTaskAccepted) {
+      decBackgroundTasks();
+    }
+
+    return isTaskAccepted;
+  }
+
   private Future<?> executeDiskStoreTask(DiskStoreTask r, ThreadPoolExecutor executor) {
     try {
       return executor.submit(r);
@@ -4628,6 +4654,18 @@ public class DiskStoreImpl implements DiskStore {
     return null;
   }
 
+  private boolean executeDiskStoreAsyncTask(DiskStoreTask r, ThreadPoolExecutor executor) {
+    try {
+      executor.execute(r);
+      return true;
+    } catch (RejectedExecutionException ex) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("Ignored compact schedule during shutdown", ex);
+      }
+    }
+    return false;
+  }
+
   private void stopDiskStoreTaskPool() {
     if (logger.isDebugEnabled()) {
       logger.debug("Stopping DiskStoreTaskPool");

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
index ba1f166..258a82e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PRHARedundancyProvider.java
@@ -1801,7 +1801,7 @@ public class PRHARedundancyProvider {
       Runnable task = new CreateMissingBucketsTask(this);
       final InternalResourceManager resourceManager =
           this.prRegion.getGemFireCache().getResourceManager();
-      resourceManager.getExecutor().submit(task);
+      resourceManager.getExecutor().execute(task);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java
index a8ad959..b22bb0e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/lru/HeapEvictor.java
@@ -423,7 +423,7 @@ public class HeapEvictor implements ResourceListener<MemoryEvent> {
         };
 
         // Submit the first pass at eviction into the pool
-        this.evictorThreadPool.submit(evictionManagerTask);
+        this.evictorThreadPool.execute(evictionManagerTask);
 
       } else {
         this.mustEvict.set(false);

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/CustomEntryConcurrentHashMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/CustomEntryConcurrentHashMap.java b/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/CustomEntryConcurrentHashMap.java
index 1e9d63e..94e55b0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/CustomEntryConcurrentHashMap.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/util/concurrent/CustomEntryConcurrentHashMap.java
@@ -1814,7 +1814,7 @@ public class CustomEntryConcurrentHashMap<K, V> extends AbstractMap<K, V>
         InternalDistributedSystem ids = InternalDistributedSystem.getConnectedInstance();
         if (ids != null) {
           try {
-            ids.getDistributionManager().getWaitingThreadPool().submit(runnable);
+            ids.getDistributionManager().getWaitingThreadPool().execute(runnable);
             submitted = true;
           } catch (RejectedExecutionException e) {
             // fall through with submitted false

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java b/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
index ab1c56a..da92f69 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/FederatingManager.java
@@ -26,7 +26,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import javax.management.InstanceNotFoundException;
 import javax.management.Notification;
 import javax.management.ObjectName;
 
@@ -196,7 +195,12 @@ public class FederatingManager extends Manager {
    */
   public void addMember(DistributedMember member) {
     GIITask giiTask = new GIITask(member);
-    submitTask(giiTask);
+    executeTask(new Runnable() {
+      @Override
+      public void run() {
+        giiTask.call();
+      }
+    });
   }
 
 
@@ -211,7 +215,7 @@ public class FederatingManager extends Manager {
    */
   public void removeMember(DistributedMember member, boolean crashed) {
     RemoveMemberTask removeTask = new RemoveMemberTask(member, crashed);
-    submitTask(removeTask);
+    executeTask(removeTask);
   }
 
   private void submitTask(Callable<DistributedMember> task) {
@@ -222,7 +226,15 @@ public class FederatingManager extends Manager {
     }
   }
 
-  private class RemoveMemberTask implements Callable<DistributedMember> {
+  private void executeTask(Runnable task) {
+    try {
+      pooledMembershipExecutor.execute(task);
+    } catch (java.util.concurrent.RejectedExecutionException ex) {
+      // Ignore, we are getting shutdown
+    }
+  }
+
+  private class RemoveMemberTask implements Runnable {
 
     private DistributedMember member;
 
@@ -233,8 +245,8 @@ public class FederatingManager extends Manager {
       this.crashed = crashed;
     }
 
-    public DistributedMember call() {
-      return removeMemberArtifacts(member, crashed);
+    public void run() {
+      removeMemberArtifacts(member, crashed);
     }
   }
 
@@ -293,7 +305,7 @@ public class FederatingManager extends Manager {
     DistributedMember member;
 
 
-    List<GIITask> giiTaskList = new ArrayList<GIITask>();
+    final List<Callable<DistributedMember>> giiTaskList = new ArrayList<>();
 
     List<Future<DistributedMember>> futureTaskList;
 

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/test/java/org/apache/geode/cache/client/internal/SingleHopClientExecutorSubmitTaskWithExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/SingleHopClientExecutorSubmitTaskWithExceptionTest.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/SingleHopClientExecutorSubmitTaskWithExceptionTest.java
new file mode 100644
index 0000000..3a9ece9
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/SingleHopClientExecutorSubmitTaskWithExceptionTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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 org.apache.geode.cache.client.internal;
+
+import java.util.concurrent.TimeUnit;
+
+import com.jayway.awaitility.Awaitility;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.SystemErrRule;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test if exceptions are logged when thread is submitted using
+ * {@code SingleHopClientExecutor#submitTask} method.
+ */
+@Category(UnitTest.class)
+public class SingleHopClientExecutorSubmitTaskWithExceptionTest {
+
+  @Rule
+  public SystemErrRule systemErrRule = new SystemErrRule().enableLog();
+
+  /**
+   * Refer: GEODE-2109 This test verifies that any exception thrown from forked thread is logged
+   * into log.
+   */
+  @Test
+  public void submittedTaskShouldLogFailure() {
+    String erroMsg = "I am expecting this to be logged";
+
+    SingleHopClientExecutor.submitTask(new Runnable() {
+      @Override
+      public void run() {
+        // test piece throwing exception
+        throw new RuntimeException(erroMsg);
+      }
+    });
+
+    /**
+     * Sometimes need to wait for more than sec as thread execution takes time.
+     */
+    Awaitility.await("Waiting for exception").atMost(60l, TimeUnit.SECONDS).until(() -> {
+      systemErrRule.getLog().contains(erroMsg);
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/test/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
index 91b3140..ed32be8 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/management/MemoryThresholdsDUnitTest.java
@@ -580,6 +580,9 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
 
   @Test
   public void testPR_RemotePutRejectionCacheClose() throws Exception {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prRemotePutRejection(true, false, false);
   }
 
@@ -595,6 +598,9 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
 
   @Test
   public void testPR_RemotePutRejectionCacheCloseWithTx() throws Exception {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prRemotePutRejection(true, false, true);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java
index 187b0bb..11efd53 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/EvictionStatsDUnitTest.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.internal.cache;
 
+import org.apache.geode.test.dunit.IgnoredException;
 import org.junit.experimental.categories.Category;
 import org.junit.Test;
 
@@ -84,6 +85,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testEntryLruLimitNDestroyLimit() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prepareScenario(EvictionAlgorithm.LRU_ENTRY);
     putData("PR1", 100);
     putData("PR2", 60);
@@ -123,6 +127,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testMemLruLimitNDestroyLimit() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prepareScenario(EvictionAlgorithm.LRU_MEMORY);
     putData("PR1", 100);
     putData("PR2", 60);
@@ -164,6 +171,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testEntryLruCounter() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prepareScenario(EvictionAlgorithm.LRU_ENTRY);
     putData("PR1", 10);
     putData("PR2", 16);
@@ -179,6 +189,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testMemLruCounter() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prepareScenario(EvictionAlgorithm.LRU_MEMORY);
     putData("PR1", 10);
     putData("PR2", 16);
@@ -194,6 +207,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testHeapLruCounter() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     prepareScenario(EvictionAlgorithm.LRU_HEAP);
     System.setProperty(HeapLRUCapacityController.TOP_UP_HEAP_EVICTION_PERCENTAGE_PROPERTY,
         Float.toString(0));
@@ -208,6 +224,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testEntryLruAllCounterMethods() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     final long ONE_MEG = 1024L * 1024L;
     createCache();
     createPartitionedRegion(true, EvictionAlgorithm.LRU_ENTRY, "PR1", 2, 1, 10000);
@@ -277,6 +296,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testEntryLRUEvictionNDestroyNNumOverflowOnDiskCount() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     final int extraEnteries = 24;
     prepareScenario(EvictionAlgorithm.LRU_ENTRY);
     putData("PR1", maxEnteries + extraEnteries);
@@ -324,6 +346,9 @@ public class EvictionStatsDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testMemLRUEvictionNDestroyNNumOverflowOnDiskCount() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     int localMaxMem = 50;
     final int extraEntries = 6;
     prepareScenario(EvictionAlgorithm.LRU_MEMORY);

http://git-wip-us.apache.org/repos/asf/geode/blob/bb3d20f4/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
index b43c0b0..cc6616a 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionEvictionDUnitTest.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.geode.test.dunit.IgnoredException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -64,6 +65,9 @@ public class PartitionedRegionEvictionDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testHeapLRUWithOverflowToDisk() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     final Host host = Host.getHost(0);
     final VM vm2 = host.getVM(2);
     final VM vm3 = host.getVM(3);
@@ -206,6 +210,9 @@ public class PartitionedRegionEvictionDUnitTest extends JUnit4CacheTestCase {
 
   @Test
   public void testHeapLRUWithLocalDestroy() {
+    // Ignore this excetion as this can happen if pool is shutting down
+    IgnoredException
+        .addIgnoredException(java.util.concurrent.RejectedExecutionException.class.getName());
     final Host host = Host.getHost(0);
     final VM vm2 = host.getVM(2);
     final VM vm3 = host.getVM(3);