Repository: incubator-geode
Updated Branches:
refs/heads/feature/GEODE-1209 18d52357d -> 0dfcde7d7
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/0dfcde7d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/0dfcde7d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/0dfcde7d
Branch: refs/heads/feature/GEODE-1209
Commit: 0dfcde7d7090225f3b044db651d485b2aeef1a96
Parents: 18d5235
Author: Anil <agingade@pivotal.io>
Authored: Tue May 3 13:52:18 2016 -0700
Committer: Anil <agingade@pivotal.io>
Committed: Thu May 5 17:41:40 2016 -0700
----------------------------------------------------------------------
.../gemfire/internal/cache/LocalRegion.java | 10 +--
.../cache/wan/AbstractGatewaySender.java | 11 ---
.../cache/xmlcache/CacheXmlGenerator.java | 8 +-
...ventQueueEvictionAndExpirationJUnitTest.java | 20 +----
.../cache30/CacheXmlGeode10DUnitTest.java | 78 ++++++++++++++++++++
5 files changed, 85 insertions(+), 42 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0dfcde7d/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/0dfcde7d/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/0dfcde7d/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..b98734d 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,12 +1522,10 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader
{
.getOrderPolicy()));
}
// eviction and expiration events
- if (asyncEventQueue.isIgnoreEvictionAndExpiration()) {
- if (generateDefaults() || asyncEventQueue.isIgnoreEvictionAndExpiration() != (GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION))
+ if (generateDefaults() || asyncEventQueue.isIgnoreEvictionAndExpiration() != (GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION))
atts.addAttribute("", "", IGNORE_EVICTION_AND_EXPIRATION, "", String.valueOf(asyncEventQueue
- .isIgnoreEvictionAndExpiration()));
- }
-
+ .isIgnoreEvictionAndExpiration()));
+
// disk-synchronous
if (generateDefaults() || asyncEventQueue.isDiskSynchronous() != GatewaySender.DEFAULT_DISK_SYNCHRONOUS)
atts.addAttribute("", "", DISK_SYNCHRONOUS, "", String.valueOf(asyncEventQueue
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0dfcde7d/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/0dfcde7d/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) {
+ }
+ }
+
}
|