You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ag...@apache.org on 2016/05/11 19:51:31 UTC

incubator-geode git commit: GEODE-1209: Added new attribute to forward eviction/expiration to AEQ. [Forced Update!]

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1209 0dfcde7d7 -> 8089d5a23 (forced update)


GEODE-1209: Added new attribute to forward eviction/expiration to AEQ.


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

Branch: refs/heads/feature/GEODE-1209
Commit: 8089d5a23616d4613f6f1525140507d715bbc90c
Parents: 18d5235
Author: Anil <ag...@pivotal.io>
Authored: Tue May 3 13:52:18 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Wed May 11 11:37:33 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/LocalRegion.java     | 10 +--
 .../cache/wan/AbstractGatewaySender.java        | 11 ---
 .../cache/xmlcache/CacheXmlGenerator.java       |  9 ++-
 ...ventQueueEvictionAndExpirationJUnitTest.java | 20 +----
 .../cache30/CacheXmlGeode10DUnitTest.java       | 78 ++++++++++++++++++++
 5 files changed, 87 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8089d5a2/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index ac3a728..fd53335 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -1147,9 +1147,7 @@ public class LocalRegion extends AbstractRegion
   @Override
   public boolean generateEventID()
   {     
-    return !(isUsedForPartitionedRegionAdmin()
-        || (isUsedForPartitionedRegionBucket() && !(((BucketRegion)this)
-            .getPartitionedRegion().getAsyncEventQueueIds().size() > 0)));
+    return !isUsedForPartitionedRegionAdmin();
   }
 
   public final Object destroy(Object key, Object aCallbackArgument)
@@ -6642,14 +6640,10 @@ public class LocalRegion extends AbstractRegion
   protected void notifyGatewaySender(EnumListenerEvent operation,
       EntryEventImpl event) {
     
-    if (this.isInternalRegion() || isPdxTypesRegion() || 
-        event.isConcurrencyConflict() /* usually concurrent cache modification problem */) { 
+    if (isPdxTypesRegion() || event.isConcurrencyConflict() /* usually concurrent cache modification problem */) { 
       return;
     }
 
-    logger.info("### notifying GW senders :" + event);
-    
-
     // Return if the inhibit all notifications flag is set
     if (event.inhibitAllNotifications()){
       if(logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8089d5a2/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 30d1fd2..713023f 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
@@ -805,7 +805,6 @@ public abstract class AbstractGatewaySender implements GatewaySender,
    * @return boolean True if the event is allowed.
    */
   private boolean checkForDistribution(EntryEventImpl event, GatewaySenderStats stats) {
-    logger.info("### isIgnoreEvictionAndExpiration :" + isIgnoreEvictionAndExpiration());
     if (event.getRegion().getDataPolicy().equals(DataPolicy.NORMAL))
     {
       return false;
@@ -846,16 +845,6 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       return;
     }
     
-<<<<<<< HEAD
-=======
-    if (getIsHDFSQueue() && event.getOperation().isEviction()) {
-      if (logger.isDebugEnabled())
-        logger.debug("Eviction event not queued: " + event);
-      stats.incEventsNotQueued();
-      return;
-    }
-    
->>>>>>> GEODE-1209: Added new attribute to forward eviction/expiration to AEQ.
     // 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

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8089d5a2/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 a4101ba..f0b1368 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
@@ -1522,16 +1522,17 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
           .getOrderPolicy()));
       }
       // eviction and expiration events
-      if (asyncEventQueue.isIgnoreEvictionAndExpiration()) {
+      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()));
+          atts.addAttribute("", "", IGNORE_EVICTION_AND_EXPIRATION, "", String.valueOf(asyncEventQueue
+              .isIgnoreEvictionAndExpiration()));
       }
-      
       // disk-synchronous
       if (generateDefaults() || asyncEventQueue.isDiskSynchronous() != GatewaySender.DEFAULT_DISK_SYNCHRONOUS)
       atts.addAttribute("", "", DISK_SYNCHRONOUS, "", String.valueOf(asyncEventQueue
           .isDiskSynchronous()));
+      
+      // AsyncEventQueue element start
       handler.startElement("", ASYNC_EVENT_QUEUE, ASYNC_EVENT_QUEUE, atts);
     
       List<GatewayEventFilter> eventFilters = asyncEventQueue.getGatewayEventFilters();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8089d5a2/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 533592c..5efac05 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
@@ -18,15 +18,9 @@ package com.gemstone.gemfire.cache.asyncqueue;
 
 import static org.junit.Assert.*;
 
-import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Set;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.junit.After;
 import org.junit.Before;
@@ -35,32 +29,22 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
-import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.EvictionAction;
 import com.gemstone.gemfire.cache.EvictionAttributes;
 import com.gemstone.gemfire.cache.ExpirationAction;
 import com.gemstone.gemfire.cache.ExpirationAttributes;
-import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionExistsException;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.TimeoutException;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
-import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.jayway.awaitility.Awaitility;
 
 import static org.mockito.Mockito.*;
 
-
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class AsyncEventQueueEvictionAndExpirationJUnitTest {
   
   private AsyncEventQueue aeq;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8089d5a2/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 57e3a13..afcb9b0 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
@@ -20,8 +20,16 @@
  */
 package com.gemstone.gemfire.cache30;
 
+import java.util.List;
+import java.util.Properties;
+
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Declarable;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
@@ -231,4 +239,74 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
       System.clearProperty("gemfire."+DistributionConfig.OFF_HEAP_MEMORY_SIZE_NAME);
     }
   }
+
+  @SuppressWarnings("rawtypes")
+  public void testAsyncEventQueueIsEnableEvictionAndExpirationAttribute() {
+
+    final String regionName = "testAsyncEventQueueIsEnableEvictionAndExpirationAttribute";
+
+    // Create AsyncEventQueue with Listener
+    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";
+    factory.create(aeqId1,listener);
+    AsyncEventQueue aeq1 = cache.getAsyncEventQueue(aeqId1);
+    assertTrue(aeq1.isIgnoreEvictionAndExpiration());
+
+    // Test by setting ignoreEvictionAndExpiration attribute value.
+    String aeqId2 = "aeqWithIgnoreEEsetToFalse";
+    factory.setIgnoreEvictionAndExpiration(false);
+    factory.create(aeqId2,listener);
+
+    AsyncEventQueue aeq2 = cache.getAsyncEventQueue(aeqId2);
+    assertFalse(aeq2.isIgnoreEvictionAndExpiration());
+
+    // Create region and set the AsyncEventQueue
+    final RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
+    attrs.addAsyncEventQueueId(aeqId2);
+
+    final Region regionBefore = cache.createRegion(regionName, attrs);
+    assertNotNull(regionBefore);
+    assertTrue(regionBefore.getAttributes().getAsyncEventQueueIds().size() == 1);
+
+
+    testXml(cache);
+
+    final Cache c = getCache();
+    assertNotNull(c);
+
+    aeq1 = c.getAsyncEventQueue(aeqId1);
+    assertTrue(aeq1.isIgnoreEvictionAndExpiration());
+
+    aeq2 = c.getAsyncEventQueue(aeqId2);
+    assertFalse(aeq2.isIgnoreEvictionAndExpiration());
+
+    final Region regionAfter = c.getRegion(regionName);
+    assertNotNull(regionAfter);
+    assertTrue(regionAfter.getAttributes().getAsyncEventQueueIds().size() == 1);
+
+    regionAfter.localDestroyRegion();
+
+    // Clear AsyncEventQueues.
+    c.close();
+  }
+
+  public static class MyAsyncEventListenerGeode10 implements AsyncEventListener, Declarable {
+
+    public boolean processEvents(List<AsyncEvent> events) {
+      return true;
+    }
+
+    public void close() {
+    }
+
+    public void init(Properties properties) {
+    }
+  }
+
 }