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

geode git commit: GEODE-1831: Function no longer gets executed twice on gateway receivers with groups

Repository: geode
Updated Branches:
  refs/heads/develop 033d14f58 -> ef96dba9a


GEODE-1831: Function no longer gets executed twice on gateway receivers with groups


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

Branch: refs/heads/develop
Commit: ef96dba9a02d3f0ffab030e9e30c27caea3b4a89
Parents: 033d14f
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Tue Dec 6 12:51:48 2016 -0800
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Wed Dec 7 09:17:19 2016 -0800

----------------------------------------------------------------------
 .../geode/internal/cache/CacheServerImpl.java   |   8 +-
 .../internal/LocatorLoadBalancingDUnitTest.java |   4 +-
 .../cache/client/internal/LocatorTestBase.java  |  26 +++-
 ...ayReceiverAutoConnectionSourceDUnitTest.java | 126 +++++++++++++++++++
 4 files changed, 153 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ef96dba9/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
index 8463e82..a3c4a93 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
@@ -690,9 +690,11 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
    */
   public String[] getCombinedGroups() {
     ArrayList<String> groupList = new ArrayList<String>();
-    for (String g : MemberAttributes.parseGroups(null, getSystem().getConfig().getGroups())) {
-      if (!groupList.contains(g)) {
-        groupList.add(g);
+    if (!this.isGatewayReceiver) {
+      for (String g : MemberAttributes.parseGroups(null, getSystem().getConfig().getGroups())) {
+        if (!groupList.contains(g)) {
+          groupList.add(g);
+        }
       }
     }
     for (String g : getGroups()) {

http://git-wip-us.apache.org/repos/asf/geode/blob/ef96dba9/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorLoadBalancingDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorLoadBalancingDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorLoadBalancingDUnitTest.java
index fffc09b..bd8143e 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorLoadBalancingDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorLoadBalancingDUnitTest.java
@@ -404,9 +404,9 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
     final ServerLoad load1 = new ServerLoad(.3f, .01f, .44f, 4564f);
     final ServerLoad load2 = new ServerLoad(23.2f, 1.1f, 22.3f, .3f);
     int serverPort1 = vm1.invoke("Start BridgeServer", () -> startBridgeServer(null, locators,
-        new String[] {REGION_NAME}, new MyLoadProbe(load1)));
+        new String[] {REGION_NAME}, new MyLoadProbe(load1), false));
     int serverPort2 = vm2.invoke("Start BridgeServer", () -> startBridgeServer(null, locators,
-        new String[] {REGION_NAME}, new MyLoadProbe(load2)));
+        new String[] {REGION_NAME}, new MyLoadProbe(load2), false));
 
     HashMap expected = new HashMap();
     ServerLocation l1 = new ServerLocation(NetworkUtils.getServerHostName(host), serverPort1);

http://git-wip-us.apache.org/repos/asf/geode/blob/ef96dba9/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorTestBase.java b/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
index d6c7b29..c3b349a 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/client/internal/LocatorTestBase.java
@@ -14,6 +14,7 @@
  */
 package org.apache.geode.cache.client.internal;
 
+import org.apache.geode.internal.lang.StringUtils;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 
 import org.apache.geode.cache.*;
@@ -161,14 +162,21 @@ public abstract class LocatorTestBase extends JUnit4DistributedTestCase {
 
   protected int startBridgeServerInVM(VM vm, final String[] groups, final String locators,
       final String[] regions) {
-    return startBridgeServerInVM(vm, groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE);
+    return startBridgeServerInVM(vm, groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE,
+        false);
+  }
+
+  protected int startBridgeServerInVM(VM vm, String[] groups, String locators,
+      boolean useGroupsProperty) {
+    return startBridgeServerInVM(vm, groups, locators, new String[] {REGION_NAME},
+        CacheServer.DEFAULT_LOAD_PROBE, useGroupsProperty);
   }
 
   protected int startBridgeServerInVM(VM vm, final String[] groups, final String locators,
-      final String[] regions, final ServerLoadProbe probe) {
+      final String[] regions, final ServerLoadProbe probe, final boolean useGroupsProperty) {
     SerializableCallable connect = new SerializableCallable("Start bridge server") {
       public Object call() throws IOException {
-        return startBridgeServer(groups, locators, regions, probe);
+        return startBridgeServer(groups, locators, regions, probe, useGroupsProperty);
       }
     };
     Integer port = (Integer) vm.invoke(connect);
@@ -181,14 +189,18 @@ public abstract class LocatorTestBase extends JUnit4DistributedTestCase {
 
   protected int startBridgeServer(final String[] groups, final String locators,
       final String[] regions) throws IOException {
-    return startBridgeServer(groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE);
+    return startBridgeServer(groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE, false);
   }
 
   protected int startBridgeServer(final String[] groups, final String locators,
-      final String[] regions, final ServerLoadProbe probe) throws IOException {
+      final String[] regions, final ServerLoadProbe probe, final boolean useGroupsProperty)
+      throws IOException {
     Properties props = new Properties();
     props.setProperty(MCAST_PORT, "0");
     props.setProperty(LOCATORS, locators);
+    if (useGroupsProperty) {
+      props.setProperty(GROUPS, StringUtils.concat(groups, ","));
+    }
     DistributedSystem ds = getSystem(props);
     Cache cache = CacheFactory.create(ds);
     AttributesFactory factory = new AttributesFactory();
@@ -201,7 +213,9 @@ public abstract class LocatorTestBase extends JUnit4DistributedTestCase {
     }
     CacheServer server = cache.addCacheServer();
     server.setPort(0);
-    server.setGroups(groups);
+    if (!useGroupsProperty) {
+      server.setGroups(groups);
+    }
     server.setLoadProbe(probe);
     server.start();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ef96dba9/geode-wan/src/test/java/org/apache/geode/cache/wan/GatewayReceiverAutoConnectionSourceDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/org/apache/geode/cache/wan/GatewayReceiverAutoConnectionSourceDUnitTest.java b/geode-wan/src/test/java/org/apache/geode/cache/wan/GatewayReceiverAutoConnectionSourceDUnitTest.java
new file mode 100644
index 0000000..fb0881c
--- /dev/null
+++ b/geode-wan/src/test/java/org/apache/geode/cache/wan/GatewayReceiverAutoConnectionSourceDUnitTest.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.cache.wan;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.client.internal.AutoConnectionSourceImpl;
+import org.apache.geode.cache.client.internal.LocatorTestBase;
+import org.apache.geode.cache.client.internal.PoolImpl;
+import org.apache.geode.distributed.internal.ServerLocation;
+import org.apache.geode.internal.AvailablePort;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.NetworkUtils;
+import org.apache.geode.test.dunit.SerializableRunnable;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+@Category({DistributedTest.class})
+public class GatewayReceiverAutoConnectionSourceDUnitTest extends LocatorTestBase {
+
+  public GatewayReceiverAutoConnectionSourceDUnitTest() {
+    super();
+  }
+
+  @Test
+  public void testBridgeServerAndGatewayReceiverClientAndServerWithoutGroup() throws Exception {
+    runBridgeServerAndGatewayReceiverTest(null, null, true);
+  }
+
+  @Test
+  public void testBridgeServerAndGatewayReceiverClientAndServerWithGroup() throws Exception {
+    String groupName = "group1";
+    runBridgeServerAndGatewayReceiverTest(new String[] {groupName}, groupName, true);
+  }
+
+  @Test
+  public void testBridgeServerAndGatewayReceiverClientWithoutGroupServerWithGroup()
+      throws Exception {
+    String groupName = "group1";
+    runBridgeServerAndGatewayReceiverTest(new String[] {groupName}, null, true);
+  }
+
+  @Test
+  public void testBridgeServerAndGatewayReceiverClientWithGroupServerWithoutGroup()
+      throws Exception {
+    String groupName = "group1";
+    runBridgeServerAndGatewayReceiverTest(null, groupName, false);
+  }
+
+  private void runBridgeServerAndGatewayReceiverTest(String[] serverGroups, String clientGroup,
+      boolean oneServerExpected) throws Exception {
+    final Host host = Host.getHost(0);
+    VM vm0 = host.getVM(0);
+    VM vm1 = host.getVM(1);
+    VM vm2 = host.getVM(2);
+
+    int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    startLocatorInVM(vm0, locatorPort, "");
+
+    String locators = NetworkUtils.getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
+
+    int serverPort = startBridgeServerInVM(vm1, serverGroups, locators, true);
+
+    addGatewayReceiverToVM(vm1);
+
+    startBridgeClientInVM(vm2, clientGroup, NetworkUtils.getServerHostName(vm0.getHost()),
+        locatorPort);
+
+    // Verify getAllServers returns a valid number of servers
+    verifyGetAllServers(vm2, REGION_NAME, serverPort, oneServerExpected);
+  }
+
+  private void addGatewayReceiverToVM(VM vm) {
+    vm.invoke(new SerializableRunnable("add GatewayReceiver") {
+      public void run() {
+        Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
+        GatewayReceiverFactory fact = cache.createGatewayReceiverFactory();
+        GatewayReceiver receiver = fact.create();
+        assertTrue(receiver.isRunning());
+      }
+    });
+  }
+
+  private void verifyGetAllServers(VM vm, final String regionName, final int serverPort,
+      final boolean oneServerExpected) {
+    vm.invoke(new SerializableRunnable("verify getAllServers") {
+      public void run() {
+        Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
+        Region region = cache.getRegion(regionName);
+        PoolImpl pool = (PoolImpl) PoolManager.find(region);
+        AutoConnectionSourceImpl connectionSource =
+            (AutoConnectionSourceImpl) pool.getConnectionSource();
+        List<ServerLocation> allServers = connectionSource.getAllServers();
+        if (oneServerExpected) {
+          // One server is expected. Assert one was returned, and its port matches the input
+          // serverPort.
+          assertEquals(1, allServers.size());
+          ServerLocation serverLocation = allServers.get(0);
+          assertEquals(serverPort, serverLocation.getPort());
+        } else {
+          // No servers are expected. Assert none were returned.
+          assertNull(allServers);
+        }
+      }
+    });
+  }
+}