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/06/18 21:05:52 UTC

[06/16] incubator-geode git commit: GEODE-1209: Changes to ignoreEvictionAndExpiration AEQ attribute based on the new proposal (changing ignoreEvictionAndExpiration to forwardExpirationDestroy).

GEODE-1209: Changes to ignoreEvictionAndExpiration AEQ attribute based on the new proposal
(changing ignoreEvictionAndExpiration to forwardExpirationDestroy).

GEODE-1209 was proposed to add new attribute ignoreEvictionAndExpiration to AsyncEventQueue.
As mentioned in the ticket due to product issue a new proposal was made to change the
functionality, to only forward expiration-destroy operation.

The changes made here replaces ignoreEvictionAndExpiration attribute to forwardExiprationDestroy.


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

Branch: refs/heads/feature/GEODE-1565
Commit: 52a13e8295bdced8bfc72ab7a710f2a4a8df0ca7
Parents: 2fe3a4b
Author: Anil <ag...@pivotal.io>
Authored: Thu Jun 16 18:07:13 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Thu Jun 16 18:16:10 2016 -0700

----------------------------------------------------------------------
 .../SharedConfigurationEndToEndDUnitTest.java   |   4 +-
 .../cache/asyncqueue/AsyncEventQueue.java       |  14 +-
 .../asyncqueue/AsyncEventQueueFactory.java      |  16 +-
 .../internal/AsyncEventQueueFactoryImpl.java    |   6 +-
 .../internal/AsyncEventQueueImpl.java           |   8 +-
 .../gemfire/cache/wan/GatewaySender.java        |   8 +-
 .../cache/wan/AbstractGatewaySender.java        |  49 ++-
 .../cache/wan/GatewaySenderAttributes.java      |  12 +-
 .../cache/xmlcache/AsyncEventQueueCreation.java |  20 +-
 .../internal/cache/xmlcache/CacheXml.java       |   4 +-
 .../cache/xmlcache/CacheXmlGenerator.java       |   6 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |  16 +-
 .../internal/cli/commands/QueueCommands.java    |  20 +-
 .../functions/AsyncEventQueueFunctionArgs.java  |  14 +-
 .../CreateAsyncEventQueueFunction.java          |  10 +-
 .../internal/cli/i18n/CliStrings.java           |   4 +-
 .../controllers/QueueCommandsController.java    |   4 +-
 .../geode.apache.org/schema/cache/cache-1.0.xsd |   2 +-
 ...ventQueueEvictionAndExpirationJUnitTest.java | 362 +++++++++++++------
 .../cache30/CacheXmlGeode10DUnitTest.java       |  23 +-
 .../cli/commands/QueueCommandsDUnitTest.java    |   4 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   4 +-
 .../cli/commands/golden-help-offline.properties |  10 +-
 23 files changed, 375 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
index 899f187..3408717 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -227,12 +227,12 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS, "4");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION, "true");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS, "true");
-      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, "true");
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, "false");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY, "1000");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, OrderPolicy.KEY.toString());
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, "true");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL, "true");
-      
+
       executeAndVerifyCommand(csb.getCommandString());
 
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
index edf887b..85ca392 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
@@ -147,13 +147,13 @@ public interface AsyncEventQueue {
    *         <code>AsyncEventQueue</code>
    */
   public GatewayEventSubstitutionFilter getGatewayEventSubstitutionFilter();
- 
+
   /**
-   * Represents if eviction and expiration events/operations are ignored (not passed)
-   * with <code>AsyncEventListener</code>.
-   * 
-   * @return boolen True if eviction and expiration operations are ignored.
+   * Represents if expiration destroy operations are forwarded (passed)
+   * to <code>AsyncEventListener</code>.
+   *
+   * @return boolean True if expiration destroy operations are forwarded.
    */
-  public boolean isIgnoreEvictionAndExpiration();
-  
+  public boolean isForwardExpirationDestroy();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
index 6294dfe..455fc14 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
@@ -171,19 +171,15 @@ public interface AsyncEventQueueFactory {
       GatewayEventSubstitutionFilter filter);
 
   /**
-   * Ignores the eviction and expiration events.
-   * By default its set to ignore eviction and expiration events (true), by
-   * setting it to false, the AEQ will receive destroy events due to eviction
-   * and expiration action.
-   * Note, setting this to false doesn't propagate invalidate events due to
-   * expiration action.
+   * Forwards destroy operations from expiration action to AsynEventQueue.
+   * By default the expiration destroy events are not added to AEQ, by
+   * setting this attribute to true adds destroy event to AEQ.
    *
-   * @param ignore 
-   *        boolean to indicate whether to ignore eviction and expiration events. 
+   * @param forward
+   *        boolean to indicate whether to forward expiration destroy events.
    */
-  public AsyncEventQueueFactory setIgnoreEvictionAndExpiration(boolean ignore);
+  public AsyncEventQueueFactory setForwardExpirationDestroy(boolean forward);
 
-  
   /**
    * Creates the <code>AsyncEventQueue</code>. It accepts Id of AsyncEventQueue
    * and instance of AsyncEventListener. Multiple queues can be created using

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
index 1ec3ba0..f658621 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
@@ -277,7 +277,7 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
     this.attrs.eventFilters = asyncQueueCreation.getGatewayEventFilters();
     this.attrs.eventSubstitutionFilter = asyncQueueCreation.getGatewayEventSubstitutionFilter();
     this.attrs.isForInternalUse = true;
-    this.attrs.ignoreEvictionAndExpiration = asyncQueueCreation.isIgnoreEvictionAndExpiration();
+    this.attrs.forwardExpirationDestroy = asyncQueueCreation.isForwardExpirationDestroy();
   }
 
   public AsyncEventQueueFactory setParallel(boolean isParallel) {
@@ -294,8 +294,8 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
   }
 
   @Override
-  public AsyncEventQueueFactory setIgnoreEvictionAndExpiration(boolean ignore) {
-    this.attrs.ignoreEvictionAndExpiration = ignore;
+  public AsyncEventQueueFactory setForwardExpirationDestroy(boolean forward) {
+    this.attrs.forwardExpirationDestroy = forward;
     return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
index 5a0b370..994bbfc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
@@ -200,9 +200,9 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
    public boolean isBucketSorted() {
     // TODO Auto-generated method stub
     return false;
-  }     
-   
-   public boolean isIgnoreEvictionAndExpiration() {
-     return ((AbstractGatewaySender)this.sender).isIgnoreEvictionAndExpiration();
+  }
+
+   public boolean isForwardExpirationDestroy() {
+     return ((AbstractGatewaySender)this.sender).isForwardExpirationDestroy();
    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
index d559a1a..04dc248 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
@@ -93,13 +93,13 @@ public interface GatewaySender {
       .getInteger(
           DistributionConfig.GEMFIRE_PREFIX + "cache.gatewaySender.defaultParallelismForReplicatedRegion",
           113).intValue();  
-  
+
   public static final int DEFAULT_DISTRIBUTED_SYSTEM_ID = -1;
 
   public static final int DEFAULT_DISPATCHER_THREADS = 5;
-  
-  public static final boolean DEFAULT_IGNORE_EVICTION_EXPIRATION = true;
-  
+
+  public static final boolean DEFAULT_FORWARD_EXPIRATION_DESTROY = false;
+
   public static final OrderPolicy DEFAULT_ORDER_POLICY = OrderPolicy.KEY;
   /**
    * The default maximum amount of memory (MB) to allow in the queue before

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
index 7e2a0af..06232a3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
@@ -137,9 +137,9 @@ public abstract class AbstractGatewaySender implements GatewaySender,
   protected List<GatewayTransportFilter> transFilters;
 
   protected List<AsyncEventListener> listeners;
-  
-  protected boolean ignoreEvictionAndExpiration;
-  
+
+  protected boolean forwardExpirationDestroy;
+
   protected GatewayEventSubstitutionFilter substitutionFilter;
   
   protected LocatorDiscoveryCallback locatorDiscoveryCallback;
@@ -272,9 +272,9 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       initializeEventIdIndex();
     }
     this.isBucketSorted = attrs.isBucketSorted();
-    this.ignoreEvictionAndExpiration = attrs.isIgnoreEvictionAndExpiration();
+    this.forwardExpirationDestroy = attrs.isForwardExpirationDestroy();
   }
-  
+
   public GatewaySenderAdvisor getSenderAdvisor() {
     return senderAdvisor;
   }
@@ -346,11 +346,11 @@ public abstract class AbstractGatewaySender implements GatewaySender,
   public boolean hasListeners() {
     return !this.listeners.isEmpty();
   }
-  
-  public boolean isIgnoreEvictionAndExpiration() {
-    return this.ignoreEvictionAndExpiration;
+
+  public boolean isForwardExpirationDestroy() {
+    return this.forwardExpirationDestroy;
   }
-  
+
   public boolean isManualStart() {
     return this.manualStart;
   }
@@ -809,26 +809,23 @@ public abstract class AbstractGatewaySender implements GatewaySender,
     {
       return false;
     }
-    
-    // Eviction and expirations are not passed to WAN.
-    // Eviction and Expiration are passed to AEQ based on its configuration.
+    // Check for eviction and expiration events.
     if (event.getOperation().isLocal() || event.getOperation().isExpiration()) {
-      // Check if its AEQ and AEQ is configured to forward eviction/expiration events.
-      if (this.isAsyncEventQueue() && !this.isIgnoreEvictionAndExpiration()) {
+      // Check if its AEQ and is configured to forward expiration destroy events.
+      if (event.getOperation().isExpiration() && this.isAsyncEventQueue() && this.isForwardExpirationDestroy()) {
         return true;
       }
       return false;
     }
-    
     return true;
   }
-  
-  
+
+
   public void distribute(EnumListenerEvent operation, EntryEventImpl event,
       List<Integer> allRemoteDSIds) {
-    
+
     final boolean isDebugEnabled = logger.isDebugEnabled();
-    
+
     // If this gateway is not running, return
     if (!isRunning()) {
       if (isDebugEnabled) {
@@ -836,20 +833,20 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       }
       return;
     }
-    
+
     final GatewaySenderStats stats = getStatistics();
     stats.incEventsReceived();
-   
+
     if (!checkForDistribution(event, stats)) {
-      getStatistics().incEventsNotQueued();
+      stats.incEventsNotQueued();
       return;
     }
-    
+
     // this filter is defined by Asif which exist in old wan too. new wan has
     // other GatewaEventFilter. Do we need to get rid of this filter. Cheetah is
-    // not cinsidering this filter
+    // not considering this filter
     if (!this.filter.enqueueEvent(event)) {
-      getStatistics().incEventsFiltered();
+      stats.incEventsFiltered();
       return;
     }
     // released by this method or transfers ownership to TmpQueueEvent
@@ -861,7 +858,7 @@ public abstract class AbstractGatewaySender implements GatewaySender,
 
     setModifiedEventId(clonedEvent);
     Object callbackArg = clonedEvent.getRawCallbackArgument();
-    
+
     if (isDebugEnabled) {
       // We can't deserialize here for logging purposes so don't
       // call getNewValue.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
index 163943f..d023dfc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
@@ -82,9 +82,9 @@ public class GatewaySenderAttributes {
   public boolean isBucketSorted = GatewaySenderAttributes.DEFAULT_IS_BUCKETSORTED;
   
   public boolean isMetaQueue = GatewaySenderAttributes.DEFAULT_IS_META_QUEUE;
-  
-  public boolean ignoreEvictionAndExpiration = GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION;
-  
+
+  public boolean forwardExpirationDestroy = GatewaySender.DEFAULT_FORWARD_EXPIRATION_DESTROY;
+
   public int getSocketBufferSize() {
     return this.socketBufferSize;
   }
@@ -194,9 +194,9 @@ public class GatewaySenderAttributes {
   public boolean isMetaQueue() {
     return this.isMetaQueue;
   }
-  
-  public boolean isIgnoreEvictionAndExpiration() {
-    return this.ignoreEvictionAndExpiration;
+
+  public boolean isForwardExpirationDestroy() {
+    return this.forwardExpirationDestroy;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
index e55ec3f..e5803c1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
@@ -43,11 +43,11 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
   private boolean isBucketSorted = false;
   private int dispatcherThreads = 1;
   private OrderPolicy orderPolicy = OrderPolicy.KEY;
-  private boolean ignoreEvictionAndExpiration = true;
-  
+  private boolean forwardExpirationDestroy = false;
+
   public AsyncEventQueueCreation() {
   }
-  
+
   public AsyncEventQueueCreation(String id, GatewaySenderAttributes senderAttrs, AsyncEventListener eventListener) {
     this.id = id;
     this.batchSize = senderAttrs.batchSize;
@@ -64,9 +64,9 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
     this.isBucketSorted = senderAttrs.isBucketSorted;
     this.gatewayEventFilters = senderAttrs.eventFilters;
     this.gatewayEventSubstitutionFilter = senderAttrs.eventSubstitutionFilter;
-    this.ignoreEvictionAndExpiration = senderAttrs.ignoreEvictionAndExpiration;
+    this.forwardExpirationDestroy = senderAttrs.forwardExpirationDestroy;
   }
-  
+
   @Override
   public AsyncEventListener getAsyncEventListener() {
     return this.asyncEventListener;
@@ -215,12 +215,12 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
     this.isBucketSorted = isBucketSorted;
   }
 
-  public void setIgnoreEvictionAndExpiration(boolean ignore) {
-    this.ignoreEvictionAndExpiration = ignore;
+  public void setForwardExpirationDestroy(boolean forward) {
+    this.forwardExpirationDestroy = forward;
   }
-  
+
   @Override
-  public boolean isIgnoreEvictionAndExpiration() {
-    return this.ignoreEvictionAndExpiration;
+  public boolean isForwardExpirationDestroy() {
+    return this.forwardExpirationDestroy;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
index 71171fd..1671d4e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
@@ -757,8 +757,8 @@ public abstract class CacheXml implements EntityResolver2, ErrorHandler {
   protected static final String ASYNC_EVENT_LISTENER = "async-event-listener";
   public static final String ASYNC_EVENT_QUEUE = "async-event-queue";
   protected static final String ASYNC_EVENT_QUEUE_IDS = "async-event-queue-ids";
-  protected static final String IGNORE_EVICTION_AND_EXPIRATION = "ignore-eviction-expiration";
-  
+  protected static final String FORWARD_EXPIRATION_DESTROY = "forward-expiration-destroy";
+
   /** The name of the <code>compressor</code> attribute */
   protected static final String COMPRESSOR = "compressor";
   /** The name of the <code>off-heap</code> attribute

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
index 17076db..8f604ba 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
@@ -1523,9 +1523,9 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
       }
       // eviction and expiration events
       if (this.version.compareTo(CacheXmlVersion.GEODE_1_0) >= 0) {
-        if (generateDefaults() || asyncEventQueue.isIgnoreEvictionAndExpiration() != (GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION))
-          atts.addAttribute("", "", IGNORE_EVICTION_AND_EXPIRATION, "", String.valueOf(asyncEventQueue
-              .isIgnoreEvictionAndExpiration()));
+        if (generateDefaults() || asyncEventQueue.isForwardExpirationDestroy() != (GatewaySender.DEFAULT_FORWARD_EXPIRATION_DESTROY))
+          atts.addAttribute("", "", FORWARD_EXPIRATION_DESTROY, "", String.valueOf(asyncEventQueue
+              .isForwardExpirationDestroy()));
       }
       // disk-synchronous
       if (generateDefaults() || asyncEventQueue.isDiskSynchronous() != GatewaySender.DEFAULT_DISK_SYNCHRONOUS)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
index 76ab0f9..0564cb5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
@@ -2312,16 +2312,16 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
                 .toLocalizedString(new Object[] { id, orderPolicy }));
       }
     }
-    
-    // forward eviction and expiration events.
-    String ignoreEvictionExpiration = atts.getValue(IGNORE_EVICTION_AND_EXPIRATION);
-    if (ignoreEvictionExpiration != null) {
-      asyncEventQueueCreation.setIgnoreEvictionAndExpiration(Boolean.parseBoolean(ignoreEvictionExpiration));
+
+    // forward expiration destroy events.
+    String forward = atts.getValue(FORWARD_EXPIRATION_DESTROY);
+    if (forward != null) {
+      asyncEventQueueCreation.setForwardExpirationDestroy(Boolean.parseBoolean(forward));
     }
-    
+
     stack.push(asyncEventQueueCreation);
   }
-  
+
   private void endAsyncEventListener() {
     Declarable d = createDeclarable();
     if (!(d instanceof AsyncEventListener)) {
@@ -2352,7 +2352,7 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
     factory.setMaximumQueueMemory(asyncEventChannelCreation.getMaximumQueueMemory());
     factory.setDispatcherThreads(asyncEventChannelCreation.getDispatcherThreads());
     factory.setOrderPolicy(asyncEventChannelCreation.getOrderPolicy());
-    factory.setIgnoreEvictionAndExpiration(asyncEventChannelCreation.isIgnoreEvictionAndExpiration());
+    factory.setForwardExpirationDestroy(asyncEventChannelCreation.isForwardExpirationDestroy());
     List<GatewayEventFilter> gatewayEventFilters = asyncEventChannelCreation.getGatewayEventFilters();
     for (GatewayEventFilter gatewayEventFilter : gatewayEventFilters) {
       factory.addGatewayEventFilter(gatewayEventFilter);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
index d84959f..c321a0d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
@@ -105,15 +105,15 @@ public class QueueCommands implements CommandMarker {
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS,
                  unspecifiedDefaultValue = "true",
                  specifiedDefaultValue = "true",
-                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP) 
+                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP)
       Boolean diskSynchronous,
-      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION,
-                 unspecifiedDefaultValue = "true",
-                 specifiedDefaultValue = "true",
-                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION__HELP) 
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY,
+                 unspecifiedDefaultValue = "false",
+                 specifiedDefaultValue = "false",
+                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY__HELP)
       Boolean ignoreEvictionAndExpiration,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY,
-                 unspecifiedDefaultValue = "100", 
+                 unspecifiedDefaultValue = "100",
                  help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP)
       int maxQueueMemory,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS,
@@ -166,13 +166,13 @@ public class QueueCommands implements CommandMarker {
         return crex.getResult();
       }
 
-      AsyncEventQueueFunctionArgs aeqArgs = new AsyncEventQueueFunctionArgs(id, parallel, 
+      AsyncEventQueueFunctionArgs aeqArgs = new AsyncEventQueueFunctionArgs(id, parallel,
           enableBatchConflation, batchSize,batchTimeInterval,
-          persistent, diskStore, diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy, 
-          gatewayEventFilters, gatewaySubstitutionListener, listener, listenerProperties, 
+          persistent, diskStore, diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy,
+          gatewayEventFilters, gatewaySubstitutionListener, listener, listenerProperties,
           ignoreEvictionAndExpiration);
 
-      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateAsyncEventQueueFunction(), 
+      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateAsyncEventQueueFunction(),
           aeqArgs, targetMembers);
 
       List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
index 2066628..a4a040b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
@@ -23,7 +23,7 @@ import java.util.Properties;
  * This class stores the arguments provided for create async event queue command.
  */
 public class AsyncEventQueueFunctionArgs implements Serializable {
-  
+
   private static final long serialVersionUID = -6524494645663740872L;
 
   private String asyncEventQueueId;
@@ -35,13 +35,13 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
   private String diskStoreName;
   private boolean diskSynchronous;
   private int maxQueueMemory;
-  private int dispatcherThreads; 
+  private int dispatcherThreads;
   private String orderPolicy;
   private String[] gatewayEventFilters;
   private String gatewaySubstitutionFilter;
   private String listenerClassName;
   private Properties listenerProperties;
-  private boolean ignoreEvictionAndExpiration;
+  private boolean forwardExpirationDestroy;
 
   public AsyncEventQueueFunctionArgs(String asyncEventQueueId,
       boolean isParallel, boolean enableBatchConflation, int batchSize,
@@ -49,7 +49,7 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
       boolean diskSynchronous, int maxQueueMemory, int dispatcherThreads,
       String orderPolicy, String[] gatewayEventFilters,
       String gatewaySubstitutionFilter, String listenerClassName,
-      Properties listenerProperties, boolean ignoreEvictionAndExpiration) {
+      Properties listenerProperties, boolean forwardExpirationDestroy) {
     this.asyncEventQueueId = asyncEventQueueId;
     this.isParallel = isParallel;
     this.enableBatchConflation = enableBatchConflation;
@@ -65,7 +65,7 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
     this.gatewaySubstitutionFilter = gatewaySubstitutionFilter;
     this.listenerClassName = listenerClassName;
     this.listenerProperties = listenerProperties;
-    this.ignoreEvictionAndExpiration = ignoreEvictionAndExpiration;
+    this.forwardExpirationDestroy = forwardExpirationDestroy;
   }
 
   public String getAsyncEventQueueId() {
@@ -128,7 +128,7 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
     return listenerProperties;
   }
 
-  public boolean isIgnoreEvictionAndExpiration() {
-    return ignoreEvictionAndExpiration;
+  public boolean isForwardExpirationDestroy() {
+    return forwardExpirationDestroy;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
index 32e8f83..c6cd8fa 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
@@ -63,7 +63,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
 
     try {
       AsyncEventQueueFunctionArgs aeqArgs =  (AsyncEventQueueFunctionArgs)context.getArguments();
-      
+
       GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
 
       DistributedMember member = cache.getDistributedSystem().getDistributedMember();
@@ -82,7 +82,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
           .setPersistent(aeqArgs.isPersistent())
           .setDiskStoreName(aeqArgs.getDiskStoreName())
           .setDiskSynchronous(aeqArgs.isDiskSynchronous())
-          .setIgnoreEvictionAndExpiration(aeqArgs.isIgnoreEvictionAndExpiration())
+          .setForwardExpirationDestroy(aeqArgs.isForwardExpirationDestroy())
           .setMaximumQueueMemory(aeqArgs.getMaxQueueMemory())
           .setDispatcherThreads(aeqArgs.getDispatcherThreads())
           .setOrderPolicy(OrderPolicy.valueOf(aeqArgs.getOrderPolicy()));
@@ -94,13 +94,13 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
           asyncEventQueueFactory.addGatewayEventFilter((GatewayEventFilter) newInstance(gatewayEventFilterKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER));
         }
       }
-      
+
       String gatewaySubstitutionFilter = aeqArgs.getGatewaySubstitutionFilter();
       if (gatewaySubstitutionFilter != null) {
         Class<?> gatewayEventSubstitutionFilterKlass = forName(gatewaySubstitutionFilter, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER);
         asyncEventQueueFactory.setGatewayEventSubstitutionListener((GatewayEventSubstitutionFilter<?,?>) newInstance(gatewayEventSubstitutionFilterKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER));
       }
-    
+
       String listenerClassName = aeqArgs.getListenerClassName();
       Object listenerInstance;
       Class<?> listenerClass = InternalDataSerializer.getCachedClass(listenerClassName);
@@ -111,7 +111,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
         if (!(listenerInstance instanceof Declarable)) {
           throw new IllegalArgumentException("Listener properties were provided, but the listener specified does not implement Declarable.");
         }
-        
+
         ((Declarable) listenerInstance).init(listenerProperties);
 
         Map<Declarable, Properties> declarablesMap = new HashMap<Declarable, Properties>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
index 787edcc..cff6535 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
@@ -426,8 +426,8 @@ public class CliStrings {
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISK_STORE__HELP = "Disk store to be used by this queue.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS = "disk-synchronous";
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP = "Whether disk writes are synchronous.";
-  public static final String CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION = "ignore-eviction-expiration";
-  public static final String CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION__HELP = "Whether to ignore eviction and expiration events.";
+  public static final String CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY = "forward-expiration-destroy";
+  public static final String CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY__HELP = "Whether to forward expiration destroy events.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY = "max-queue-memory";
   public static final String CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP = "Maximum amount of memory, in megabytes, that the queue can consume before overflowing to disk.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER = "gateway-event-filter";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
index 9367612..387ea4e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
@@ -58,7 +58,7 @@ public class QueueCommandsController extends AbstractCommandsController {
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, defaultValue = "false") final Boolean persistent,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE, required = false) final String diskStore,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS, defaultValue = "true") final Boolean diskSynchronous,
-                                      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, defaultValue = "true") final Boolean isIgnoreEvictionAndExpiration,
+                                      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, defaultValue = "false") final Boolean forwardExpirationDestroy,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY, defaultValue = "100") final Integer maxQueueMemory,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS, defaultValue = "1") final Integer dispatcherThreads,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, defaultValue = "KEY") final String orderPolicy,
@@ -83,7 +83,7 @@ public class QueueCommandsController extends AbstractCommandsController {
 
     command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL, String.valueOf(Boolean.TRUE.equals(parallel)));
     command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION, String.valueOf(Boolean.TRUE.equals(enableBatchConflation)));
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, String.valueOf(isIgnoreEvictionAndExpiration));
+    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, String.valueOf(forwardExpirationDestroy));
 
     if (hasValue(batchSize)) {
       command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE, String.valueOf(batchSize));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
----------------------------------------------------------------------
diff --git a/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd b/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
index 688ff1f..452baa3 100755
--- a/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
+++ b/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
@@ -254,7 +254,7 @@ declarative caching XML file elements unless indicated otherwise.
             <xsd:attribute name="disk-synchronous" type="xsd:boolean" use="optional" />
             <xsd:attribute name="dispatcher-threads" type="xsd:string" use="optional" />
             <xsd:attribute name="order-policy" type="xsd:string" use="optional" />
-            <xsd:attribute default="true" name="ignore-eviction-expiration" type="xsd:boolean" use="optional" />
+            <xsd:attribute default="false" name="forward-expiration-destroy" type="xsd:boolean" use="optional" />
           </xsd:complexType>
         </xsd:element>
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
index 9add46b..c37810f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
@@ -17,9 +17,14 @@
 package com.gemstone.gemfire.cache.asyncqueue;
 
 import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.jayway.awaitility.Awaitility;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -32,6 +37,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import static com.gemstone.gemfire.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
@@ -41,12 +47,16 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
   
   private AsyncEventQueue aeq;
   private Cache cache;
+  private Region region;
+  String aeqId;
+  List<AsyncEvent> events = new ArrayList<AsyncEvent>();
   
   @Rule 
   public TestName name = new TestName();
   
   @Before
-  public void getCache() {
+  public void setup() {
+    events.clear();
     try {
        cache = CacheFactory.getAnyInstance();
     } catch (Exception e) {
@@ -55,10 +65,11 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
     if (null == cache) {
       cache = (GemFireCacheImpl) new CacheFactory().set(MCAST_PORT, "0").create();
     }
+    aeqId = name.getMethodName();
   }
 
   @After
-  public void destroyCache() {
+  public void tearDown() {
     if (cache != null && !cache.isClosed()) {
       cache.close();
       cache = null;
@@ -67,19 +78,19 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
 
   
   @Test
-  public void isIgnoreEvictionAndExpirationAttributeTrueByDefault() {
+  public void isForwardExpirationDestroyAttributeFalseByDefault() {
     AsyncEventListener al = mock(AsyncEventListener.class);
     aeq = cache.createAsyncEventQueueFactory().create("aeq", al);
     // Test for default value of isIgnoreEvictionAndExpiration setting.
-    assertTrue(aeq.isIgnoreEvictionAndExpiration());
+    assertFalse(aeq.isForwardExpirationDestroy());
   }
   
   @Test
-  public void canSetFalseForIgnoreEvictionAndExpiration() {
+  public void canSetTrueForForwardExpirationDestroy() {
     AsyncEventListener al = mock(AsyncEventListener.class);
-    aeq = cache.createAsyncEventQueueFactory().setIgnoreEvictionAndExpiration(false).create("aeq", al);
+    aeq = cache.createAsyncEventQueueFactory().setForwardExpirationDestroy(true).create("aeq", al);
     // Test for default value of isIgnoreEvictionAndExpiration setting.
-    assertFalse(aeq.isIgnoreEvictionAndExpiration());
+    assertTrue(aeq.isForwardExpirationDestroy());
   }
   
   
@@ -87,182 +98,298 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
   public void evictionDestroyOpEventsNotPropogatedByDefault() {
     // For Replicated Region with eviction-destroy op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, false /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
   public void evictionDestroyOpEventsNotPropogatedByDefaultForPR() {
     // For PR with eviction-destroy op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, false /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */,
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+   // Validate events that are not queued.
+   // This guarantees that eviction/expiration is performed and events are
+   // sent all the way to Gateway.
+   // In case of eviction one event is evicted that should not be queued.
+   Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+   // The AQListner should get expected events.
+   Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+    
   }
 
   @Test
   public void expirationDestroyOpEventsNotPropogatedByDefault() {
     // For Replicated Region with expiration-destroy op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
   public void expirationDestroyOpEventsNotPropogatedByDefaultForPR() {
     // For PR with expiration-destroy op.
-    // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 2. Two for create and none for eviction destroy.    
+    createRegionAeqAndPopulate(true /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
   public void expirationInvalidOpEventsNotPropogatedByDefault() {
     // For Replicated Region with expiration-invalid op.
-    // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 2. Two for create and none for eviction destroy.    
+    createRegionAeqAndPopulate(false /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+
+    LocalRegion lr = (LocalRegion)region;
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
+    
   }
 
   @Test
   public void expirationInvalidOpEventsNotPropogatedByDefaultForPR() {
     // For Replicated Region with expiration-invalid op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+
+    LocalRegion lr = (LocalRegion)region;
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
   }
   
   @Test
-  public void evictionPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void evictionNotPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with eviction-destroy op.
-    // Number of expected events 3. Two for create and One for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */, 
-        3 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 2. Two for create and none for eviction destroy.
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
+    
   }
 
   @Test
-  public void evictionPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void evictionNotPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with eviction-destroy op.
-    // Number of expected events 3. Two for create and One for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */, 
-        3 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 3. Two for create and none for eviction destroy.
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
+
   }
 
   @Test
-  public void overflowNotPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void overflowNotPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with eviction-overflow op.
     // Number of expected events 2. Two for create and non for eviction overflow.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, true /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, true /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+    
+    // Wait for region to evict/expire events.
+    LocalRegion lr = (LocalRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, lr.getDiskRegion().getStats().getNumOverflowOnDisk()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
   }
 
   @Test
-  public void overflowNotPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void overflowNotPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with eviction-overflow op.
     // Number of expected events 2. Two for create and non for eviction overflow.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, true /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, true /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+    
+    // Wait for region to evict/expire events.
+    PartitionedRegion pr = (PartitionedRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, pr.getDiskRegionStats().getNumOverflowOnDisk()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
-  public void expirationDestroyPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void expirationDestroyPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with expiration-destroy op.
     // Number of expected events 4. Two for create and Two for expiration destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */,  
-        4 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);    
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */); 
+    
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+    
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, getEventsReceived(aeqId)));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, events.size()));
+    
+    assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
+        
   }
 
   @Test
-  public void expirationDestroyPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void expirationDestroyPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with expiration-destroy op.
     // Number of expected events 4. Two for create and Two for expiration destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */,  
-        4 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);    
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */); 
+    
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+    
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, getEventsReceived(aeqId)));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, events.size()));
+    
+    assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
+
   }
 
   @Test
-  public void expirationInvalidateNotPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void expirationInvalidateNotPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with expiration-invalidate op.
     // Currently invalidate event callbacks are not made to GateWay sender.
     // Invalidates are not sent to AEQ.
     // Number of expected events 2. None for expiration invalidate.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */,
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+    
+ // Wait for region to evict/expire events.
+    LocalRegion lr = (LocalRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+    
   }
 
   @Test
-  public void expirationInvalidateNotPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void expirationInvalidateNotPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with expiration-invalidate op.
     // Currently invalidate event callbacks are not made to GateWay sender.
     // Invalidates are not sent to AEQ.
     // Number of expected events 2. None for expiration invalidate.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */,
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+    
+    // Wait for region to evict/expire events.
+    LocalRegion lr = (LocalRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   
-  
-  private void createPopulateAndVerifyEvents(boolean isPR, boolean ignoreEvictionExpiration, int expectedEvents, boolean eviction, boolean evictionOverflow, 
-      boolean expirationDestroy, boolean expirationInvalidate, boolean checkForDestroyOp, boolean checkForInvalidateOp) {
-    
+
+  private void createRegionAeqAndPopulate(boolean isPR, boolean forwardExpirationDestroy, boolean eviction, boolean evictionOverflow, 
+      boolean expirationDestroy, boolean expirationInvalidate) {
     // String aeqId = "AEQTest";
     String aeqId = name.getMethodName();
-    
-    // To store AEQ events for validation.
-    List<AsyncEvent> events = new ArrayList<AsyncEvent>();
-    
+
     // Create AEQ
-    createAsyncEventQueue(aeqId, ignoreEvictionExpiration, events);    
-    
-    // Create region with eviction/expiration
-    Region r = createRegion("ReplicatedRegionForAEQ", isPR, aeqId, eviction, evictionOverflow, expirationDestroy, expirationInvalidate);
-    
-    // Populate region with two entires.
-    r.put("Key-1", "Value-1");
-    r.put("Key-2", "Value-2");
+    createAsyncEventQueue(aeqId, forwardExpirationDestroy, events);
+
+    region = createRegion("ReplicatedRegionForAEQ", isPR, aeqId, eviction, evictionOverflow, expirationDestroy, expirationInvalidate);
     
-    // The AQListner should get two events. One for create, one for destroy.
-    Awaitility.await().atMost(100, TimeUnit.SECONDS).until(() -> {return (events.size() == expectedEvents);});
+    // Populate region with two entries.
+    region.put("Key-1", "Value-1");
+    region.put("Key-2", "Value-2");
     
-    // Check for the expected operation.
-    if (checkForDestroyOp) {
-      assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
-    }
+    try {
+      Thread.sleep(2000);
+    } catch (Exception ex) {}
+  }
 
-    if (checkForInvalidateOp) {
-      assertTrue("Invalidate event not arrived", checkForOperation(events, true, false));
-    }
-    
-    // Test complete. Destroy region.
-    r.destroyRegion();
+
+  private void waitForAEQEventsNotQueued() {
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {return (getEventsNotQueuedSize(aeqId) >= 1);});
   }
+  
 
   private boolean checkForOperation(List<AsyncEvent> events, boolean invalidate, boolean destroy) {
     boolean found = false;
@@ -278,11 +405,24 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
     }
     return found;
   }
+  
+  public int getEventsNotQueuedSize(String aeqId) {
+    AsyncEventQueueImpl aeq  = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
+    AbstractGatewaySender sender = (AbstractGatewaySender)aeq.getSender();
+    return sender.getStatistics().getEventsNotQueued();
+  }
+
+
+  public int getEventsReceived(String aeqId) {
+    AsyncEventQueueImpl aeq  = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
+    AbstractGatewaySender sender = (AbstractGatewaySender)aeq.getSender();
+    return sender.getStatistics().getEventsReceived();
+  }
 
-  private void createAsyncEventQueue(String id, boolean ignoreEvictionExpiration, List<AsyncEvent> storeEvents) {
+  private void createAsyncEventQueue(String id, boolean forwardExpirationDestroy, List<AsyncEvent> storeEvents) {
     AsyncEventListener al = this.createAsyncListener(storeEvents);
     aeq = cache.createAsyncEventQueueFactory().setParallel(false)
-        .setIgnoreEvictionAndExpiration(ignoreEvictionExpiration)
+        .setForwardExpirationDestroy(forwardExpirationDestroy)
         .setBatchSize(1).setBatchTimeInterval(1).create(id, al);
   }
   
@@ -318,16 +458,16 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
       
       @Override
       public void close() {
-        // TODO Auto-generated method stub
       }
 
       @Override
       public boolean processEvents(List<AsyncEvent> arg0) {
-        System.out.println("AEQ Listener.process()");
-        new Exception("Stack trace for AsyncEventQueue").printStackTrace();
-        // TODO Auto-generated method stub
-        aeList.addAll(arg0);
-        System.out.println("AEQ Event :" + arg0);
+        try {
+          synchronized(aeList) {
+            aeList.add(arg0.get(0));
+          }
+        } catch (Exception ex) {
+        }
         return true;
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
index b5f7f45..6f6e9e9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
@@ -254,7 +254,7 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
 
   @SuppressWarnings("rawtypes")
   @Test
-  public void testAsyncEventQueueIsEnableEvictionAndExpirationAttribute() {
+  public void testAsyncEventQueueIsForwardExpirationDestroyAttribute() {
 
     final String regionName = "testAsyncEventQueueIsEnableEvictionAndExpirationAttribute";
 
@@ -262,22 +262,22 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
     final CacheCreation cache = new CacheCreation();
     AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
 
-    
+
     AsyncEventListener listener = new MyAsyncEventListenerGeode10();
 
-    // Test for default ignoreEvictionAndExpiration attribute value (which is true)
-    String aeqId1 = "aeqWithDefaultIgnoreEE";
+    // Test for default forwardExpirationDestroy attribute value (which is false)
+    String aeqId1 = "aeqWithDefaultFED";
     factory.create(aeqId1,listener);
     AsyncEventQueue aeq1 = cache.getAsyncEventQueue(aeqId1);
-    assertTrue(aeq1.isIgnoreEvictionAndExpiration());
+    assertFalse(aeq1.isForwardExpirationDestroy());
 
-    // Test by setting ignoreEvictionAndExpiration attribute value.
-    String aeqId2 = "aeqWithIgnoreEEsetToFalse";
-    factory.setIgnoreEvictionAndExpiration(false);
+    // Test by setting forwardExpirationDestroy attribute value.
+    String aeqId2 = "aeqWithFEDsetToTrue";
+    factory.setForwardExpirationDestroy(true);
     factory.create(aeqId2,listener);
 
     AsyncEventQueue aeq2 = cache.getAsyncEventQueue(aeqId2);
-    assertFalse(aeq2.isIgnoreEvictionAndExpiration());
+    assertTrue(aeq2.isForwardExpirationDestroy());
 
     // Create region and set the AsyncEventQueue
     final RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
@@ -287,17 +287,16 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
     assertNotNull(regionBefore);
     assertTrue(regionBefore.getAttributes().getAsyncEventQueueIds().size() == 1);
 
-
     testXml(cache);
 
     final Cache c = getCache();
     assertNotNull(c);
 
     aeq1 = c.getAsyncEventQueue(aeqId1);
-    assertTrue(aeq1.isIgnoreEvictionAndExpiration());
+    assertFalse(aeq1.isForwardExpirationDestroy());
 
     aeq2 = c.getAsyncEventQueue(aeqId2);
-    assertFalse(aeq2.isIgnoreEvictionAndExpiration());
+    assertTrue(aeq2.isForwardExpirationDestroy());
 
     final Region regionAfter = c.getRegion(regionName);
     assertNotNull(regionAfter);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
index 152e55a..86c61b6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
@@ -167,7 +167,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER,
         "com.qcdunit.QueueCommandsDUnitTestHelper");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS, "false");
-    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, "false");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, "true");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
         "com.qcdunit.QueueCommandsDUnitTestHelper");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE, "param1");
@@ -209,7 +209,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
         assertEquals(queue.getGatewayEventSubstitutionFilter().getClass().getName(),
             "com.qcdunit.QueueCommandsDUnitTestHelper");
         assertEquals(queue.isDiskSynchronous(), false);
-        assertEquals(queue.isIgnoreEvictionAndExpiration(), false);
+        assertEquals(queue.isForwardExpirationDestroy(), true);
         assertEquals(queue.getAsyncEventListener().getClass().getName(), "com.qcdunit.QueueCommandsDUnitTestHelper");
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index 2147219..3c85d5d 100755
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@ -256,7 +256,6 @@ com/gemstone/gemfire/internal/admin/CompoundRegionSnapshot,true,6295026394298398
 com/gemstone/gemfire/internal/admin/StatAlert,true,5725457607122449170,definitionId:int,time:java/util/Date,values:java/lang/Number[]
 com/gemstone/gemfire/internal/admin/remote/DistributionLocatorId,true,6587390186971937865,bindAddress:java/lang/String,host:java/net/InetAddress,hostnameForClients:java/lang/String,peerLocator:boolean,port:int,serverLocator:boolean
 com/gemstone/gemfire/internal/admin/remote/EntryValueNodeImpl,false,fields:com/gemstone/gemfire/internal/admin/remote/EntryValueNodeImpl[],name:java/lang/String,primitive:boolean,primitiveVal:java/lang/Object,type:java/lang/String
-com/gemstone/gemfire/internal/cache/AbstractRegionMap$1,false,this$0:com/gemstone/gemfire/internal/cache/AbstractRegionMap
 com/gemstone/gemfire/internal/cache/BackupLock,false,backupDone:java/util/concurrent/locks/Condition,backupThread:java/lang/Thread,isBackingUp:boolean
 com/gemstone/gemfire/internal/cache/BucketAdvisor$SetFromMap,true,2454657854757543876,m:java/util/Map
 com/gemstone/gemfire/internal/cache/BucketNotFoundException,false
@@ -281,7 +280,6 @@ com/gemstone/gemfire/internal/cache/ForceableLinkedBlockingQueue,true,-690393397
 com/gemstone/gemfire/internal/cache/GemFireCacheImpl$3,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
 com/gemstone/gemfire/internal/cache/GemFireCacheImpl$4,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
 com/gemstone/gemfire/internal/cache/GemFireCacheImpl$5,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
-com/gemstone/gemfire/internal/cache/GemFireCacheImpl$6,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
 com/gemstone/gemfire/internal/cache/IdentityArrayList,true,449125332499184497,size:int,wrapped:boolean
 com/gemstone/gemfire/internal/cache/IncomingGatewayStatus,true,-4579815367602658353,_memberId:java/lang/String,_socketAddress:java/net/InetAddress,_socketPort:int
 com/gemstone/gemfire/internal/cache/InitialImageOperation$GIIStatus,false
@@ -677,7 +675,7 @@ com/gemstone/gemfire/management/internal/cli/exceptions/CliException,false
 com/gemstone/gemfire/management/internal/cli/exceptions/CreateSubregionException,true,4387344870743824916
 com/gemstone/gemfire/management/internal/cli/exceptions/IndexNotFoundException,true,1,indexName:java/lang/String,message:java/lang/String
 com/gemstone/gemfire/management/internal/cli/functions/AlterRuntimeConfigFunction,true,1
-com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs,true,-6524494645663740872,asyncEventQueueId:java/lang/String,batchSize:int,batchTimeInterval:int,diskStoreName:java/lang/String,diskSynchronous:boolean,dispatcherThreads:int,enableBatchConflation:boolean,gatewayEventFilters:java/lang/String[],gatewaySubstitutionFilter:java/lang/String,ignoreEvictionAndExpiration:boolean,isParallel:boolean,listenerClassName:java/lang/String,listenerProperties:java/util/Properties,maxQueueMemory:int,orderPolicy:java/lang/String,persistent:boolean
+com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs,true,-6524494645663740872,asyncEventQueueId:java/lang/String,batchSize:int,batchTimeInterval:int,diskStoreName:java/lang/String,diskSynchronous:boolean,dispatcherThreads:int,enableBatchConflation:boolean,forwardExpirationDestroy:boolean,gatewayEventFilters:java/lang/String[],gatewaySubstitutionFilter:java/lang/String,isParallel:boolean,listenerClassName:java/lang/String,listenerProperties:java/util/Properties,maxQueueMemory:int,orderPolicy:java/lang/String,persistent:boolean
 com/gemstone/gemfire/management/internal/cli/functions/ChangeLogLevelFunction,true,1
 com/gemstone/gemfire/management/internal/cli/functions/CloseDurableClientFunction,true,1
 com/gemstone/gemfire/management/internal/cli/functions/CloseDurableCqFunction,true,1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
index 3c0d388..0cd7a07 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
@@ -540,7 +540,7 @@ SYNTAX\n\
 \ \ \ \ create async-event-queue --id=value --listener=value [--group=value(nullvalue)*]\n\
 \ \ \ \ [--parallel(=value)?] [--enable-batch-conflation(=value)?] [--batch-size=value]\n\
 \ \ \ \ [--batch-time-interval=value] [--persistent(=value)?] [--disk-store=value]\n\
-\ \ \ \ [--disk-synchronous(=value)?] [--ignore-eviction-expiration(=value)?]\n\
+\ \ \ \ [--disk-synchronous(=value)?] [--forward-expiration-destroy(=value)?]\n\
 \ \ \ \ [--max-queue-memory=value] [--dispatcher-threads=value] [--order-policy=value]\n\
 \ \ \ \ [--gateway-event-filter=value(,value)*] [--gateway-event-substitution-filter=value]\n\
 \ \ \ \ [--listener-param=value(,value)*]\n\
@@ -583,11 +583,11 @@ PARAMETERS\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is specified without value): true\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is not specified): true\n\
-\ \ \ \ ignore-eviction-expiration\n\
-\ \ \ \ \ \ \ \ Whether to ignore eviction and expiration events.\n\
+\ \ \ \ forward-expiration-destroy\n\
+\ \ \ \ \ \ \ \ Whether to forward expiration destroy events.\n\
 \ \ \ \ \ \ \ \ Required: false\n\
-\ \ \ \ \ \ \ \ Default (if the parameter is specified without value): true\n\
-\ \ \ \ \ \ \ \ Default (if the parameter is not specified): true\n\
+\ \ \ \ \ \ \ \ Default (if the parameter is specified without value): false\n\
+\ \ \ \ \ \ \ \ Default (if the parameter is not specified): false\n\
 \ \ \ \ max-queue-memory\n\
 \ \ \ \ \ \ \ \ Maximum amount of memory, in megabytes, that the queue can consume before overflowing to\n\
 \ \ \ \ \ \ \ \ disk.\n\