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

[46/55] [abbrv] incubator-geode git commit: GEODE-1471: GatewayEventFilter callbacks are now invoked on AsyncEventQueues

GEODE-1471: GatewayEventFilter callbacks are now invoked on AsyncEventQueues


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

Branch: refs/heads/feature/GEODE-1372
Commit: 001a4e167d287e66a944edf45ac9591b09104954
Parents: c742c4e
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Thu Jun 2 18:25:05 2016 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Mon Jun 6 09:56:48 2016 -0700

----------------------------------------------------------------------
 .../cache/xmlcache/AsyncEventQueueCreation.java |  3 +-
 .../cache/wan/MyAsyncEventListener.java         |  7 +-
 .../cache/wan/MyGatewayEventFilter.java         | 69 ++++++++++++++++++++
 .../AsyncEventQueueValidationsJUnitTest.java    | 56 ++++++++++++++++
 ...ntQueueConfiguredFromXmlUsesFilter.cache.xml | 40 ++++++++++++
 ...ntQueueConfiguredFromXmlUsesFilter.cache.xml | 40 ++++++++++++
 6 files changed, 213 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/001a4e16/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 4c2943e..e55ec3f 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
@@ -61,7 +61,8 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
     this.dispatcherThreads = senderAttrs.dispatcherThreads;
     this.orderPolicy = senderAttrs.policy;
     this.asyncEventListener = eventListener;
-    this.isBucketSorted = senderAttrs.isBucketSorted; 
+    this.isBucketSorted = senderAttrs.isBucketSorted;
+    this.gatewayEventFilters = senderAttrs.eventFilters;
     this.gatewayEventSubstitutionFilter = senderAttrs.eventSubstitutionFilter;
     this.ignoreEvictionAndExpiration = senderAttrs.ignoreEvictionAndExpiration;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/001a4e16/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyAsyncEventListener.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyAsyncEventListener.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyAsyncEventListener.java
index f2401a5..9e8357d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyAsyncEventListener.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyAsyncEventListener.java
@@ -18,13 +18,15 @@ package com.gemstone.gemfire.internal.cache.wan;
 
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.gemstone.gemfire.cache.Declarable;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventListener;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 
-public class MyAsyncEventListener implements AsyncEventListener {
+public class MyAsyncEventListener implements AsyncEventListener, Declarable {
 
   private final Map eventsMap;
 
@@ -50,4 +52,7 @@ public class MyAsyncEventListener implements AsyncEventListener {
 
   public void close() {
   }
+
+  @Override
+  public void init(Properties props) {}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/001a4e16/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyGatewayEventFilter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyGatewayEventFilter.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyGatewayEventFilter.java
new file mode 100644
index 0000000..39766e9
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/MyGatewayEventFilter.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache.wan;
+
+import com.gemstone.gemfire.cache.Declarable;
+import com.gemstone.gemfire.cache.wan.GatewayEventFilter;
+import com.gemstone.gemfire.cache.wan.GatewayQueueEvent;
+
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// Note: This class is used by AsyncEventQueueValidationsJUnitTest testAsyncEventQueueConfiguredFromXmlUsesFilter
+public class MyGatewayEventFilter implements GatewayEventFilter, Declarable {
+
+  private AtomicInteger beforeEnqueueInvocations = new AtomicInteger();
+
+  private AtomicInteger beforeTransmitInvocations = new AtomicInteger();
+
+  private AtomicInteger afterAcknowledgementInvocations = new AtomicInteger();
+
+  @Override
+  public boolean beforeEnqueue(GatewayQueueEvent event) {
+    beforeEnqueueInvocations.incrementAndGet();
+    return true;
+  }
+
+  @Override
+  public boolean beforeTransmit(GatewayQueueEvent event) {
+    beforeTransmitInvocations.incrementAndGet();
+    return true;
+  }
+
+  @Override
+  public void afterAcknowledgement(GatewayQueueEvent event) {
+    afterAcknowledgementInvocations.incrementAndGet();
+  }
+
+  public int getBeforeEnqueueInvocations() {
+    return this.beforeEnqueueInvocations.get();
+  }
+
+  public int getBeforeTransmitInvocations() {
+    return this.beforeTransmitInvocations.get();
+  }
+
+  public int getAfterAcknowledgementInvocations() {
+    return this.afterAcknowledgementInvocations.get();
+  }
+
+  @Override
+  public void init(Properties props) {}
+
+  @Override
+  public void close() {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/001a4e16/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
index 2a7e57f..3055c8e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.java
@@ -21,14 +21,29 @@ package com.gemstone.gemfire.internal.cache.wan.asyncqueue;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
+import com.gemstone.gemfire.cache.wan.GatewayEventFilter;
 import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
 import com.gemstone.gemfire.internal.cache.wan.AsyncEventQueueConfigurationException;
+import com.gemstone.gemfire.internal.cache.wan.MyGatewayEventFilter;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
+import com.jayway.awaitility.Awaitility;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
 
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static com.gemstone.gemfire.distributed.DistributedSystemConfigProperties.CACHE_XML_FILE;
 import static com.gemstone.gemfire.distributed.DistributedSystemConfigProperties.MCAST_PORT;
+import static junitparams.JUnitParamsRunner.$;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -36,10 +51,18 @@ import static org.junit.Assert.fail;
  *
  */
 @Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class AsyncEventQueueValidationsJUnitTest {
 
   private Cache cache;
 
+  @After
+  public void closeCache() {
+    if(this.cache != null) {
+      this.cache.close();
+    }
+  }
+
   @Test
   public void testConcurrentParallelAsyncEventQueueAttributesWrongDispatcherThreads() {
     cache = new CacheFactory().set(MCAST_PORT, "0").create();
@@ -72,4 +95,37 @@ public class AsyncEventQueueValidationsJUnitTest {
     }
   }
 
+  @Test
+  @Parameters(method = "getCacheXmlFileBaseNames")
+  public void testAsyncEventQueueConfiguredFromXmlUsesFilter(String cacheXmlFileBaseName) {
+    // Create cache with xml
+    String cacheXmlFileName = TestUtil.getResourcePath(getClass(), getClass().getSimpleName() + "." + cacheXmlFileBaseName + ".cache.xml");
+    cache = new CacheFactory().set(MCAST_PORT, "0").set(CACHE_XML_FILE, cacheXmlFileName).create();
+
+    // Get region and do puts
+    Region region = cache.getRegion(cacheXmlFileBaseName);
+    int numPuts = 10;
+    for (int i=0; i<numPuts; i++) {
+      region.put(i,i);
+    }
+
+    // Get AsyncEventQueue and GatewayEventFilter
+    AsyncEventQueue aeq = cache.getAsyncEventQueue(cacheXmlFileBaseName);
+    List<GatewayEventFilter> filters = aeq.getGatewayEventFilters();
+    assertTrue(filters.size() == 1);
+    MyGatewayEventFilter filter = (MyGatewayEventFilter) filters.get(0);
+
+    // Validate filter callbacks were invoked
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> filter.getBeforeEnqueueInvocations() == numPuts);
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> filter.getBeforeTransmitInvocations() == numPuts);
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> filter.getAfterAcknowledgementInvocations() == numPuts);
+  }
+
+  private final Object[] getCacheXmlFileBaseNames() {
+    return $(
+        new Object[] { "testSerialAsyncEventQueueConfiguredFromXmlUsesFilter" },
+        new Object[] { "testParallelAsyncEventQueueConfiguredFromXmlUsesFilter" }
+    );
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/001a4e16/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testParallelAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testParallelAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml b/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testParallelAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml
new file mode 100755
index 0000000..9d8ed7f
--- /dev/null
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testParallelAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!DOCTYPE cache PUBLIC
+  "-//GemStone Systems, Inc.//GemFire Declarative Caching 8.0//EN"
+  "http://www.gemstone.com/dtd/cache8_0.dtd">
+  
+<cache>
+
+  <async-event-queue id="testParallelAsyncEventQueueConfiguredFromXmlUsesFilter" parallel="true">
+    <gateway-event-filter>
+      <class-name>com.gemstone.gemfire.internal.cache.wan.MyGatewayEventFilter</class-name>
+    </gateway-event-filter>
+    <async-event-listener>
+      <class-name>com.gemstone.gemfire.internal.cache.wan.MyAsyncEventListener</class-name>
+    </async-event-listener>
+  </async-event-queue>
+
+  <region name="testParallelAsyncEventQueueConfiguredFromXmlUsesFilter" refid="PARTITION">
+    <region-attributes async-event-queue-ids="testParallelAsyncEventQueueConfiguredFromXmlUsesFilter"/>
+  </region>
+
+</cache>
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/001a4e16/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testSerialAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testSerialAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml b/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testSerialAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml
new file mode 100755
index 0000000..1361036
--- /dev/null
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/internal/cache/wan/asyncqueue/AsyncEventQueueValidationsJUnitTest.testSerialAsyncEventQueueConfiguredFromXmlUsesFilter.cache.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!DOCTYPE cache PUBLIC
+  "-//GemStone Systems, Inc.//GemFire Declarative Caching 8.0//EN"
+  "http://www.gemstone.com/dtd/cache8_0.dtd">
+  
+<cache>
+
+  <async-event-queue id="testSerialAsyncEventQueueConfiguredFromXmlUsesFilter" parallel="false">
+    <gateway-event-filter>
+      <class-name>com.gemstone.gemfire.internal.cache.wan.MyGatewayEventFilter</class-name>
+    </gateway-event-filter>
+    <async-event-listener>
+      <class-name>com.gemstone.gemfire.internal.cache.wan.MyAsyncEventListener</class-name>
+    </async-event-listener>
+  </async-event-queue>
+
+  <region name="testSerialAsyncEventQueueConfiguredFromXmlUsesFilter" refid="PARTITION">
+    <region-attributes async-event-queue-ids="testSerialAsyncEventQueueConfiguredFromXmlUsesFilter"/>
+  </region>
+
+</cache>
+