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/05/20 17:10:26 UTC

[01/50] [abbrv] incubator-geode git commit: Revert "GEODE-1376: Cleaned up server port to be '0'."

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-835 29861e4fd -> 7e559224a


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
index 176fbea..9ef87d2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.cache.client.internal;
 
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Assert;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.locator.ClientConnectionRequest;
@@ -39,31 +50,24 @@ import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.jayway.awaitility.Awaitility;
-import org.junit.Assert;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.InetAddress;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  *
  */
 public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
-
+  
   /**
    * The number of connections that we can be off by in the balancing tests
    * We need this little fudge factor, because the locator can receive an update
    * from the bridge server after it has made incremented its counter for a client
    * connection, but the client hasn't connected yet. This wipes out the estimation
    * on the locator.  This means that we may be slighly off in our balance.
-   * <p>
+   * 
    * TODO grid fix this hole in the locator.
    */
   private static final int ALLOWABLE_ERROR_IN_COUNT = 1;
@@ -74,144 +78,149 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
   }
 
   /**
-   * Test the locator discovers a bridge server and is initialized with
+   * Test the locator discovers a bridge server and is initialized with 
    * the correct load for that bridge server.
    */
   public void testDiscovery() {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-
+    VM vm2 = host.getVM(2);
+//    vm0.invoke(new SerializableRunnable() {
+//      public void run() {
+//        System.setProperty("gemfire.DistributionAdvisor.VERBOSE", "true");
+//      }
+//    });
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
-
+    startLocatorInVM(vm0, locatorPort, "");
+    
     String locators = getLocatorString(host, locatorPort);
-
-    int serverPort = vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-
+    
+    int serverPort = startBridgeServerInVM(vm1, new String[] {"a", "b"},  locators);
+    
     ServerLoad expectedLoad = new ServerLoad(0f, 1 / 800.0f, 0f, 1f);
     ServerLocation expectedLocation = new ServerLocation(NetworkUtils.getServerHostName(vm0
         .getHost()), serverPort);
     Map expected = new HashMap();
     expected.put(expectedLocation, expectedLoad);
-
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
-    int serverPort2 = vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-
+    
+    checkLocatorLoad(vm0, expected);
+    
+    int serverPort2 = startBridgeServerInVM(vm2, new String[] {"a", "b"},  locators);
+    
     ServerLocation expectedLocation2 = new ServerLocation(NetworkUtils.getServerHostName(vm0
         .getHost()), serverPort2);
-
+    
     expected.put(expectedLocation2, expectedLoad);
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
+    checkLocatorLoad(vm0, expected);
   }
-
+  
   /**
    * Test that the locator will properly estimate the load for servers when
-   * it receives connection requests.
+   * it receives connection requests. 
    */
-  public void testEstimation() throws IOException, ClassNotFoundException {
+  public void testEstimation() throws UnknownHostException, IOException, ClassNotFoundException {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
+    startLocatorInVM(vm0, locatorPort, "");
     String locators = getLocatorString(host, locatorPort);
-
-    int serverPort = vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-
-    ServerLoad expectedLoad = new ServerLoad(2 / 800f, 1 / 800.0f, 0f, 1f);
+    
+    int serverPort = startBridgeServerInVM(vm1, new String[] {"a", "b"},  locators);
+    
+    ServerLoad expectedLoad = new ServerLoad(2/800f, 1 / 800.0f, 0f, 1f);
     ServerLocation expectedLocation = new ServerLocation(NetworkUtils.getServerHostName(host), serverPort);
     Map expected = new HashMap();
     expected.put(expectedLocation, expectedLoad);
-
+    
     ClientConnectionResponse response;
     response = (ClientConnectionResponse) TcpClient.requestToServer(InetAddress
-            .getByName(NetworkUtils.getServerHostName(host)), locatorPort,
+        .getByName(NetworkUtils.getServerHostName(host)), locatorPort,
         new ClientConnectionRequest(Collections.EMPTY_SET, null), 10000);
     Assert.assertEquals(expectedLocation, response.getServer());
-
+    
     response = (ClientConnectionResponse) TcpClient.requestToServer(InetAddress
-            .getByName(NetworkUtils.getServerHostName(host)), locatorPort,
+        .getByName(NetworkUtils.getServerHostName(host)), locatorPort,
         new ClientConnectionRequest(Collections.EMPTY_SET, null), 10000, true);
     Assert.assertEquals(expectedLocation, response.getServer());
-
+    
     //we expect that the connection load load will be 2 * the loadPerConnection
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
+    checkLocatorLoad(vm0, expected);
+    
     QueueConnectionResponse response2;
     response2 = (QueueConnectionResponse) TcpClient.requestToServer(InetAddress
-            .getByName(NetworkUtils.getServerHostName(host)), locatorPort,
+        .getByName(NetworkUtils.getServerHostName(host)), locatorPort,
         new QueueConnectionRequest(null, 2,
             Collections.EMPTY_SET, null, false), 10000, true);
     Assert.assertEquals(Collections.singletonList(expectedLocation), response2.getServers());
-
+    
     response2 = (QueueConnectionResponse) TcpClient
         .requestToServer(InetAddress.getByName(NetworkUtils.getServerHostName(host)),
             locatorPort, new QueueConnectionRequest(null, 5, Collections.EMPTY_SET, null,
                 false), 10000, true);
-
+    
     Assert.assertEquals(Collections.singletonList(expectedLocation), response2.getServers());
 
     //we expect that the queue load will increase by 2
     expectedLoad.setSubscriptionConnectionLoad(2f);
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
+    checkLocatorLoad(vm0, expected);
   }
-
+  
   /**
    * Test to make sure the bridge servers communicate
    * their updated load to the controller when the load
    * on the bridge server changes.
-   *
-   * @throws Exception
+   * @throws Exception 
    */
   public void testLoadMessaging() 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);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
+    startLocatorInVM(vm0, locatorPort, "");
     String locators = getLocatorString(host, locatorPort);
-
-    final int serverPort = vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-
+    
+    final int serverPort = startBridgeServerInVM(vm1, new String[] {"a", "b"},  locators);
+    
     //We expect 0 load
     Map expected = new HashMap();
     ServerLocation expectedLocation = new ServerLocation(NetworkUtils.getServerHostName(host), serverPort);
     ServerLoad expectedLoad = new ServerLoad(0f, 1 / 800.0f, 0f, 1f);
     expected.put(expectedLocation, expectedLoad);
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
+    checkLocatorLoad(vm0, expected);
+    
     PoolFactoryImpl pf = new PoolFactoryImpl(null);
     pf.addServer(NetworkUtils.getServerHostName(host), serverPort);
     pf.setMinConnections(8);
     pf.setMaxConnections(8);
     pf.setSubscriptionEnabled(true);
-    vm2.invoke("StartBridgeClient", () -> startBridgeClient(pf.getPoolAttributes(), new String[] { REGION_NAME }));
-
+    startBridgeClientInVM(vm2, pf.getPoolAttributes(), new String[] {REGION_NAME});
+    
     //We expect 8 client to server connections. The queue requires
     //an additional client to server connection, but that shouldn't show up here.
-    expectedLoad = new ServerLoad(8 / 800f, 1 / 800.0f, 1f, 1f);
+    expectedLoad = new ServerLoad(8/800f, 1 / 800.0f, 1f, 1f);
     expected.put(expectedLocation, expectedLoad);
-
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
+    
+    
+    checkLocatorLoad(vm0, expected);
+    
     stopBridgeMemberVM(vm2);
-
+    
     //Now we expect 0 load
     expectedLoad = new ServerLoad(0f, 1 / 800.0f, 0f, 1f);
     expected.put(expectedLocation, expectedLoad);
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
+    checkLocatorLoad(vm0, expected);
   }
-
+  
   /**
    * Test to make sure that the locator
    * balancing load between two servers.
-   *
-   * @throws Exception
+   * @throws Exception 
    */
   public void testBalancing() throws Exception {
     final Host host = Host.getHost(0);
@@ -219,60 +228,87 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
+    startLocatorInVM(vm0, locatorPort, "");
     String locators = getLocatorString(host, locatorPort);
-
-    vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-    vm2.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-
+    
+    startBridgeServerInVM(vm1, new String[] {"a", "b"},  locators);
+    startBridgeServerInVM(vm2, new String[] {"a", "b"},  locators);
+    
     PoolFactoryImpl pf = new PoolFactoryImpl(null);
     pf.addLocator(NetworkUtils.getServerHostName(host), locatorPort);
     pf.setMinConnections(80);
     pf.setMaxConnections(80);
     pf.setSubscriptionEnabled(false);
     pf.setIdleTimeout(-1);
-    vm3.invoke("StartBridgeClient", () -> startBridgeClient(pf.getPoolAttributes(), new String[] { REGION_NAME }));
-
-    vm3.invoke("waitForPrefilledConnections", () -> waitForPrefilledConnections(80));
-
-    vm1.invoke("check connection count", () -> checkConnectionCount(40));
-    vm2.invoke("check connection count", () -> checkConnectionCount(40));
+    startBridgeClientInVM(vm3, pf.getPoolAttributes(), new String[] {REGION_NAME});
+    
+    waitForPrefilledConnections(vm3, 80);
+    
+    checkConnectionCount(vm1, 40);
+    checkConnectionCount(vm2, 40);
   }
 
-  private void checkConnectionCount(final int count) {
-    Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-    final CacheServerImpl server = (CacheServerImpl)
-        cache.getCacheServers().get(0);
-    Awaitility.await().pollDelay(100, TimeUnit.MILLISECONDS).pollInterval(100, TimeUnit.MILLISECONDS)
-        .timeout(300, TimeUnit.SECONDS).until(() -> {
-      int sz = server.getAcceptor().getStats().getCurrentClientConnections();
-      if (Math.abs(sz - count) <= ALLOWABLE_ERROR_IN_COUNT) {
-        return true;
+  private void checkConnectionCount(VM vm, final int count) {
+    SerializableRunnableIF checkConnectionCount = new SerializableRunnable("checkConnectionCount") {
+      public void run() {
+        Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
+        final CacheServerImpl server = (CacheServerImpl)
+            cache.getCacheServers().get(0);
+        WaitCriterion wc = new WaitCriterion() {
+          String excuse;
+          public boolean done() {
+            int sz = server.getAcceptor().getStats()
+                .getCurrentClientConnections();
+            if (Math.abs(sz - count) <= ALLOWABLE_ERROR_IN_COUNT) {
+              return true;
+            }
+            excuse = "Found " + sz + " connections, expected " + count;
+            return false;
+          }
+          public String description() {
+            return excuse;
+          }
+        };
+        Wait.waitForCriterion(wc, 5 * 60 * 1000, 1000, true);
       }
-      System.out.println("Found " + sz + " connections, expected " + count);
-      return false;
-    });
+    };
+    
+    vm.invoke(checkConnectionCount);
   }
-
-  private void waitForPrefilledConnections(final int count) throws Exception {
-    waitForPrefilledConnections(count, POOL_NAME);
+  
+  private void waitForPrefilledConnections(VM vm, final int count) throws Exception {
+    waitForPrefilledConnections(vm, count, POOL_NAME);
   }
 
-  private void waitForPrefilledConnections(final int count, final String poolName) throws Exception {
-    final PoolImpl pool = (PoolImpl) PoolManager.getAll().get(poolName);
-    Awaitility.await().pollDelay(100, TimeUnit.MILLISECONDS).pollInterval(100, TimeUnit.MILLISECONDS)
-        .timeout(300, TimeUnit.SECONDS).until(() -> pool.getConnectionCount() >= count);
+  private void waitForPrefilledConnections(VM vm, final int count, final String poolName) throws Exception {
+    SerializableRunnable runnable = new SerializableRunnable("waitForPrefilledConnections") {
+      public void run() {
+        final PoolImpl pool = (PoolImpl) PoolManager.getAll().get(poolName);
+        WaitCriterion ev = new WaitCriterion() {
+          public boolean done() {
+            return pool.getConnectionCount() >= count; 
+          }
+          public String description() {
+            return "connection count never reached " + count;
+          }
+        };
+        Wait.waitForCriterion(ev, MAX_WAIT, 200, true);
+      }
+    };
+    if(vm == null) {
+      runnable.run();
+    } else {
+      vm.invoke(runnable);
+    }
   }
-
-  /**
-   * Test that the locator balances load between
+  
+  /** Test that the locator balances load between
    * three servers with intersecting server groups.
    * Server:    1       2       3
    * Groups:    a       a,b     b
-   *
-   * @throws Exception
+   * @throws Exception 
    */
   public void testIntersectingServerGroups() throws Exception {
     final Host host = Host.getHost(0);
@@ -280,158 +316,175 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
+    startLocatorInVM(vm0, locatorPort, "");
     String locators = getLocatorString(host, locatorPort);
-
-    int serverPort1 = vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a" }, locators));
-    vm2.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "a", "b" }, locators));
-    vm3.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "b" }, locators));
-
+    
+    int serverPort1 = startBridgeServerInVM(vm1, new String[] {"a"},  locators);
+    startBridgeServerInVM(vm2, new String[] {"a", "b"},  locators);
+    startBridgeServerInVM(vm3, new String[] {"b"},  locators);
+    
     PoolFactoryImpl pf = new PoolFactoryImpl(null);
     pf.addLocator(NetworkUtils.getServerHostName(host), locatorPort);
     pf.setMinConnections(12);
     pf.setSubscriptionEnabled(false);
     pf.setServerGroup("a");
     pf.setIdleTimeout(-1);
-    startBridgeClient(pf.getPoolAttributes(), new String[] { REGION_NAME });
-    waitForPrefilledConnections(12);
-
-    vm1.invoke("Check Connection Count", () -> checkConnectionCount(6));
-    vm2.invoke("Check Connection Count", () -> checkConnectionCount(6));
-    vm3.invoke("Check Connection Count", () -> checkConnectionCount(0));
-
+    startBridgeClientInVM(null, pf.getPoolAttributes(), new String[] {REGION_NAME});
+    waitForPrefilledConnections(null, 12);
+    
+    checkConnectionCount(vm1, 6);
+    checkConnectionCount(vm2, 6);
+    checkConnectionCount(vm3, 0);
+    
     LogWriterUtils.getLogWriter().info("pool1 prefilled");
-
+    
     PoolFactoryImpl pf2 = (PoolFactoryImpl) PoolManager.createFactory();
     pf2.init(pf.getPoolAttributes());
     pf2.setServerGroup("b");
-    PoolImpl pool2 = (PoolImpl) pf2.create("testPool2");
-    waitForPrefilledConnections(12, "testPool2");
+    PoolImpl pool2= (PoolImpl) pf2.create("testPool2");
+    waitForPrefilledConnections(null, 12, "testPool2");
 
     // The load will not be perfect, because we created all of the connections
     //for group A first.
-    vm1.invoke("Check Connection Count", () -> checkConnectionCount(6));
-    vm2.invoke("Check Connection Count", () -> checkConnectionCount(9));
-    vm3.invoke("Check Connection Count", () -> checkConnectionCount(9));
-
+    checkConnectionCount(vm1, 6);
+    checkConnectionCount(vm2, 9);
+    checkConnectionCount(vm3, 9);
+    
     LogWriterUtils.getLogWriter().info("pool2 prefilled");
-
+    
     ServerLocation location1 = new ServerLocation(NetworkUtils.getServerHostName(host), serverPort1);
     PoolImpl pool1 = (PoolImpl) PoolManager.getAll().get(POOL_NAME);
     Assert.assertEquals("a", pool1.getServerGroup());
-
+    
     //Use up all of the pooled connections on pool1, and acquire 3 more
-    for (int i = 0; i < 15; i++) {
+    for(int i = 0; i < 15; i++) {
       pool1.acquireConnection();
     }
-
+    
     LogWriterUtils.getLogWriter().info("aquired 15 connections in pool1");
-
+    
     //now the load should be equal
-    vm1.invoke("Check Connection Count", () -> checkConnectionCount(9));
-    vm2.invoke("Check Connection Count", () -> checkConnectionCount(9));
-    vm3.invoke("Check Connection Count", () -> checkConnectionCount(9));
-
+    checkConnectionCount(vm1, 9);
+    checkConnectionCount(vm2, 9);
+    checkConnectionCount(vm3, 9);
+    
     //use up all of the pooled connections on pool2
-    for (int i = 0; i < 12; i++) {
+    for(int i = 0; i < 12; i++) {
       pool2.acquireConnection();
     }
-
+    
     LogWriterUtils.getLogWriter().info("aquired 12 connections in pool2");
-
+    
     //interleave creating connections in both pools
-    for (int i = 0; i < 6; i++) {
+    for(int i = 0; i < 6; i++) {
       pool1.acquireConnection();
       pool2.acquireConnection();
     }
-
+    
     LogWriterUtils.getLogWriter().info("interleaved 6 connections from pool1 with 6 connections from pool2");
-
+    
     //The load should still be balanced
-    vm1.invoke("Check Connection Count", () -> checkConnectionCount(13));
-    vm2.invoke("Check Connection Count", () -> checkConnectionCount(13));
-    vm3.invoke("Check Connection Count", () -> checkConnectionCount(13));
-
+    checkConnectionCount(vm1, 13);
+    checkConnectionCount(vm2, 13);
+    checkConnectionCount(vm3, 13);
+    
   }
-
+  
   public void testCustomLoadProbe() throws Exception {
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
-    //    VM vm3 = host.getVM(3);
-
+//    VM vm3 = host.getVM(3);
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
+    startLocatorInVM(vm0, locatorPort, "");
     String locators = getLocatorString(host, locatorPort);
-
-    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)));
-    int serverPort2 = vm2.invoke("Start BridgeServer", () -> startBridgeServer(null, locators, new String[] { REGION_NAME }, new MyLoadProbe(load2)));
-
+    
+    ServerLoad load1= new ServerLoad(.3f, .01f, .44f, 4564f);
+    ServerLoad load2= new ServerLoad(23.2f, 1.1f, 22.3f, .3f);
+    int serverPort1 = startBridgeServerInVM(vm1, null, locators, new String[] {REGION_NAME}, new MyLoadProbe(load1 ));
+    int serverPort2 = startBridgeServerInVM(vm2, null, locators, new String[] {REGION_NAME}, new MyLoadProbe(load2 ));
+    
     HashMap expected = new HashMap();
     ServerLocation l1 = new ServerLocation(NetworkUtils.getServerHostName(host), serverPort1);
     ServerLocation l2 = new ServerLocation(NetworkUtils.getServerHostName(host), serverPort2);
     expected.put(l1, load1);
     expected.put(l2, load2);
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
+    checkLocatorLoad(vm0, expected);
+    
     load1.setConnectionLoad(25f);
-    vm1.invoke("changeLoad", () -> changeLoad(load1));
+    changeLoad(vm1, load1);
     load2.setSubscriptionConnectionLoad(3.5f);
-    vm2.invoke("changeLoad", () -> changeLoad(load2));
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
-    final ServerLoad load1Updated = new ServerLoad(1f, .1f, 0f, 1f);
-    final ServerLoad load2Updated = new ServerLoad(2f, 5f, 0f, 2f);
-    expected.put(l1, load1Updated);
-    expected.put(l2, load2Updated);
-    vm1.invoke("changeLoad", () -> changeLoad(load1Updated));
-    vm2.invoke("changeLoad", () -> changeLoad(load2Updated));
-    vm0.invoke("check Locator Load", () -> checkLocatorLoad(expected));
-
+    changeLoad(vm2, load2);
+    checkLocatorLoad(vm0, expected);
+    
+    load1 = new ServerLoad(1f, .1f, 0f, 1f);
+    load2 = new ServerLoad(2f, 5f, 0f, 2f);
+    expected.put(l1, load1);
+    expected.put(l2, load2);
+    changeLoad(vm1, load1);
+    changeLoad(vm2, load2);
+    checkLocatorLoad(vm0, expected);
+    
     PoolFactoryImpl pf = new PoolFactoryImpl(null);
     pf.addLocator(NetworkUtils.getServerHostName(host), locatorPort);
     pf.setMinConnections(20);
     pf.setSubscriptionEnabled(true);
     pf.setIdleTimeout(-1);
-    startBridgeClient(pf.getPoolAttributes(), new String[] { REGION_NAME });
-    waitForPrefilledConnections(20);
-
+    startBridgeClientInVM(null, pf.getPoolAttributes(), new String[] {REGION_NAME});
+    waitForPrefilledConnections(null, 20);
+    
     //The first 10 connection should to go vm1, then 1 to vm2, then another 9 to vm1
     //because have unequal values for loadPerConnection
-    vm1.invoke("Check Connection Count", () -> checkConnectionCount(19));
-    vm2.invoke("Check Connection Count", () -> checkConnectionCount(1));
+    checkConnectionCount(vm1, 19);
+    checkConnectionCount(vm2, 1);
   }
-
-  public void checkLocatorLoad(final Map expected) {
-    List locators = Locator.getLocators();
-    Assert.assertEquals(1, locators.size());
-    InternalLocator locator = (InternalLocator) locators.get(0);
-    final ServerLocator sl = locator.getServerLocatorAdvisee();
-    InternalLogWriter log = new LocalLogWriter(InternalLogWriter.FINEST_LEVEL, System.out);
-    sl.getDistributionAdvisor().dumpProfiles("PROFILES= ");
-    Awaitility.await().pollDelay(100, TimeUnit.MILLISECONDS).pollInterval(100, TimeUnit.MILLISECONDS)
-        .timeout(300, TimeUnit.SECONDS).until(() -> expected.equals(sl.getLoadMap()));
+  
+  public void checkLocatorLoad(VM vm, final Map expected) {
+    vm.invoke(new SerializableRunnable() {
+      public void run() {
+        List locators = Locator.getLocators();
+        Assert.assertEquals(1, locators.size());
+        InternalLocator locator = (InternalLocator) locators.get(0);
+        final ServerLocator sl = locator.getServerLocatorAdvisee();
+        InternalLogWriter log = new LocalLogWriter(InternalLogWriter.FINEST_LEVEL, System.out);
+        sl.getDistributionAdvisor().dumpProfiles("PROFILES= ");
+        WaitCriterion ev = new WaitCriterion() {
+          public boolean done() {
+            return expected.equals(sl.getLoadMap());
+          }
+          public String description() {
+            return "load map never became equal to " + expected;
+          }
+        };
+        Wait.waitForCriterion(ev, MAX_WAIT, 200, true);
+      }
+    });
   }
-
-  private void changeLoad(final ServerLoad newLoad) {
-    Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-    CacheServer server = cache.getCacheServers().get(0);
-    MyLoadProbe probe = (MyLoadProbe) server.getLoadProbe();
-    probe.setLoad(newLoad);
+  
+  private void changeLoad(VM vm, final ServerLoad newLoad) {
+    vm.invoke(new SerializableRunnable() {
+
+      public void run() {
+        Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
+        CacheServer server = (CacheServer) cache.getCacheServers().get(0);
+        MyLoadProbe probe = (MyLoadProbe) server.getLoadProbe();
+        probe.setLoad(newLoad);
+      }
+      
+    });
   }
-
+  
   private static class MyLoadProbe extends ServerLoadProbeAdapter implements Serializable {
     private ServerLoad load;
-
+    
     public MyLoadProbe(ServerLoad load) {
       this.load = load;
     }
-
+    
     public ServerLoad getLoad(ServerMetrics metrics) {
       float connectionLoad = load.getConnectionLoad()
           + metrics.getConnectionCount() * load.getLoadPerConnection();
@@ -440,7 +493,7 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
       return new ServerLoad(connectionLoad, load.getLoadPerConnection(),
           queueLoad, load.getLoadPerSubscriptionConnection());
     }
-
+    
     public void setLoad(ServerLoad load) {
       this.load = load;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
index af5ba9c..2207e1d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
@@ -16,35 +16,52 @@
  */
 package com.gemstone.gemfire.cache.client.internal;
 
-import com.gemstone.gemfire.cache.*;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.PoolFactoryImpl;
-import com.gemstone.gemfire.test.dunit.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.util.*;
+import com.gemstone.gemfire.test.dunit.Assert;
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
 
 /**
  *
  */
-public abstract class LocatorTestBase extends DistributedTestCase {
+public abstract class LocatorTestBase  extends DistributedTestCase {
   protected static final String CACHE_KEY = "CACHE";
   protected static final String LOCATOR_KEY = "LOCATOR";
   protected static final String REGION_NAME = "A_REGION";
   protected static final String POOL_NAME = "daPool";
   protected static final Object CALLBACK_KEY = "callback";
-  /**
-   * A map for storing temporary objects in a remote VM so that they can be used
+  /** A map for storing temporary objects in a remote VM so that they can be used
    * between calls. Cleared after each test.
    */
   protected static final HashMap remoteObjects = new HashMap();
@@ -52,211 +69,264 @@ public abstract class LocatorTestBase extends DistributedTestCase {
   public LocatorTestBase(String name) {
     super(name);
   }
-
+  
   @Override
   public final void preTearDown() throws Exception {
-
+    
     SerializableRunnable tearDown = new SerializableRunnable("tearDown") {
       public void run() {
         Locator locator = (Locator) remoteObjects.get(LOCATOR_KEY);
-        if (locator != null) {
+        if(locator != null) {
           try {
             locator.stop();
-          } catch (Exception e) {
+          } catch(Exception e) {
             //do nothing
           }
         }
-
+        
         Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-        if (cache != null) {
+        if(cache != null) {
           try {
             cache.close();
-          } catch (Exception e) {
+          } catch(Exception e) {
             //do nothing
           }
         }
         remoteObjects.clear();
-
+        
       }
     };
     //We seem to like leaving the DS open if we can for
     //speed, but lets at least destroy our cache and locator.
     Invoke.invokeInEveryVM(tearDown);
     tearDown.run();
-
+    
     postTearDownLocatorTestBase();
   }
-
+  
   protected void postTearDownLocatorTestBase() throws Exception {
   }
+  
+  protected void startLocatorInVM(final VM vm, final int locatorPort, final String otherLocators) {
+    vm.invoke(new SerializableRunnable("Create Locator") {
 
-  protected void startLocator(final Host vmHost, final int locatorPort, final String otherLocators) {
-    disconnectFromDS();
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
-    props.setProperty(DistributionConfig.LOCATORS_NAME, otherLocators);
-    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
-    props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
-    File logFile = new File(getUniqueName() + "-locator" + locatorPort + ".log");
-    try {
-      InetAddress bindAddress = InetAddress.getByName(NetworkUtils.getServerHostName(vmHost));
-      Locator locator = Locator.startLocatorAndDS(locatorPort, logFile, bindAddress, props);
-      remoteObjects.put(LOCATOR_KEY, locator);
-    } catch (UnknownHostException uhe) {
-      Assert.fail("While resolving bind address ", uhe);
-    } catch (IOException ex) {
-      Assert.fail("While starting locator on port " + locatorPort, ex);
-    }
+      final String testName= getUniqueName();
+      public void run() {
+        disconnectFromDS();
+        Properties props = new Properties();
+        props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+        props.setProperty(DistributionConfig.LOCATORS_NAME, otherLocators);
+        props.setProperty(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
+        props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+        try {
+          File logFile = new File(testName + "-locator" + locatorPort
+              + ".log");
+          InetAddress bindAddr = null;
+          try {
+            bindAddr = InetAddress.getByName(NetworkUtils.getServerHostName(vm.getHost()));
+          } catch (UnknownHostException uhe) {
+            Assert.fail("While resolving bind address ", uhe);
+          }
+          Locator locator = Locator.startLocatorAndDS(locatorPort, logFile, bindAddr, props);
+          remoteObjects.put(LOCATOR_KEY, locator);
+        } catch (IOException ex) {
+          Assert.fail("While starting locator on port " + locatorPort, ex);
+        }
+      }
+    });
   }
-
-  protected void stopLocator() {
-    Locator locator = (Locator) remoteObjects.remove(LOCATOR_KEY);
-    locator.stop();
+  
+  
+  
+  protected void stopLocatorInVM(VM vm) {
+    vm.invoke(new SerializableRunnable("Stop Locator") {
+      public void run() {
+        Locator locator = (Locator) remoteObjects.remove(LOCATOR_KEY);
+        locator.stop();
+      }
+    });
   }
-
-  protected int startBridgeServer(String[] groups, String locators) throws IOException {
-    return startBridgeServer(groups, locators, new String[] { REGION_NAME });
+  
+  protected int startBridgeServerInVM(VM vm, String[] groups, String locators) {
+    return startBridgeServerInVM(vm, groups, locators, new String[] {REGION_NAME});
   }
+  
+  protected int addCacheServerInVM(VM vm, final String[] groups) {
+    SerializableCallable connect =
+      new SerializableCallable("Add Bridge server") {
 
-  protected int addCacheServer(final String[] groups) throws IOException {
-    Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-    CacheServer server = cache.addCacheServer();
-    server.setPort(0);
-    server.setGroups(groups);
-    server.start();
-    return new Integer(server.getPort());
+      public Object call() throws Exception {
+        Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
+        CacheServer server = cache.addCacheServer();
+        final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+        server.setPort(serverPort);
+        server.setGroups(groups);
+        server.start();
+        return new Integer(serverPort);
+      }
+    };
+    Integer port = (Integer) vm.invoke(connect);
+    return port.intValue();
   }
-
-  protected int startBridgeServer(final String[] groups, final String locators, final String[] regions) throws IOException {
-    return startBridgeServer(groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE);
+  
+  protected int startBridgeServerInVM(VM vm, final String[] groups, final String locators, final String[] regions) {
+    return startBridgeServerInVM(vm, groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE);
   }
-
-  protected int startBridgeServer(final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe) throws IOException {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
-    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
-    DistributedSystem ds = getSystem(props);
-    Cache cache = CacheFactory.create(ds);
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setEnableBridgeConflation(true);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attributes = factory.create();
-    for (int i = 0; i < regions.length; i++) {
-      cache.createRegion(regions[i], attributes);
-    }
-    CacheServer server = cache.addCacheServer();
-    server.setPort(0);
-    server.setGroups(groups);
-    server.setLoadProbe(probe);
-    server.start();
-
-    remoteObjects.put(CACHE_KEY, cache);
-
-    return new Integer(server.getPort());
+  
+  protected int startBridgeServerInVM(VM vm, final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe) {
+    SerializableCallable connect =
+      new SerializableCallable("Start bridge server") {
+          public Object call() throws IOException  {
+            Properties props = new Properties();
+            props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+            props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+            DistributedSystem ds = getSystem(props);
+            Cache cache = CacheFactory.create(ds);
+            AttributesFactory factory = new AttributesFactory();
+            factory.setScope(Scope.DISTRIBUTED_ACK);
+            factory.setEnableBridgeConflation(true);
+            factory.setDataPolicy(DataPolicy.REPLICATE);
+            RegionAttributes attrs = factory.create();
+            for(int i = 0; i < regions.length; i++) {
+              cache.createRegion(regions[i], attrs);
+            }
+            CacheServer server = cache.addCacheServer();
+            final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+            server.setPort(serverPort);
+            server.setGroups(groups);
+            server.setLoadProbe(probe);
+            server.start();
+            
+            remoteObjects.put(CACHE_KEY, cache);
+            
+            return new Integer(serverPort);
+          }
+        };
+    Integer port = (Integer) vm.invoke(connect);
+    return port.intValue();
   }
-
-  protected int startBridgeServerWithEmbeddedLocator(final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe)
-      throws IOException {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
-    props.setProperty(DistributionConfig.START_LOCATOR_NAME, locators);
-    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
-    DistributedSystem ds = getSystem(props);
-    Cache cache = CacheFactory.create(ds);
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setEnableBridgeConflation(true);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attrs = factory.create();
-    for (int i = 0; i < regions.length; i++) {
-      cache.createRegion(regions[i], attrs);
-    }
-    CacheServer server = cache.addCacheServer();
-    server.setGroups(groups);
-    server.setLoadProbe(probe);
-    server.setPort(0);
-    server.start();
-
-    remoteObjects.put(CACHE_KEY, cache);
-
-    return new Integer(server.getPort());
+  
+  protected int startBridgeServerWithEmbeddedLocator(VM vm, final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe) {
+    SerializableCallable connect =
+      new SerializableCallable("Start bridge server") {
+          public Object call() throws IOException  {
+            Properties props = new Properties();
+            props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+            props.setProperty(DistributionConfig.START_LOCATOR_NAME, locators);
+            props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+            DistributedSystem ds = getSystem(props);
+            Cache cache = CacheFactory.create(ds);
+            AttributesFactory factory = new AttributesFactory();
+            factory.setScope(Scope.DISTRIBUTED_ACK);
+            factory.setEnableBridgeConflation(true);
+            factory.setDataPolicy(DataPolicy.REPLICATE);
+            RegionAttributes attrs = factory.create();
+            for(int i = 0; i < regions.length; i++) {
+              cache.createRegion(regions[i], attrs);
+            }
+            CacheServer server = cache.addCacheServer();
+            server.setGroups(groups);
+            server.setLoadProbe(probe);
+            final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+            server.setPort(serverPort);
+            server.start();
+            
+            remoteObjects.put(CACHE_KEY, cache);
+            
+            return new Integer(serverPort);
+          }
+        };
+    Integer port = (Integer) vm.invoke(connect);
+    return port.intValue();
   }
-
-  protected void startBridgeClient(final String group, final String host, final int port) throws Exception {
-    startBridgeClient(group, host, port, new String[] { REGION_NAME });
+  
+  protected void  startBridgeClientInVM(VM vm, final String group, final String host, final int port) throws Exception {
+    startBridgeClientInVM(vm, group, host, port, new String[] {REGION_NAME});
   }
+  
 
-  protected void startBridgeClient(final String group, final String host, final int port, final String[] regions) throws Exception {
+  protected void startBridgeClientInVM(VM vm, final String group, final String host, final int port, final String[] regions) throws Exception {
     PoolFactoryImpl pf = new PoolFactoryImpl(null);
     pf.addLocator(host, port)
-        .setServerGroup(group)
-        .setPingInterval(200)
-        .setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1);
-    startBridgeClient(pf.getPoolAttributes(), regions);
+    .setServerGroup(group)
+    .setPingInterval(200)
+    .setSubscriptionEnabled(true)
+    .setSubscriptionRedundancy(-1);
+    startBridgeClientInVM(vm, pf.getPoolAttributes(), regions);
   }
-
-  protected void startBridgeClient(final Pool pool, final String[] regions) throws Exception {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    DistributedSystem ds = getSystem(props);
-    Cache cache = CacheFactory.create(ds);
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    //            factory.setEnableBridgeConflation(true);
-    //            factory.setDataPolicy(DataPolicy.NORMAL);
-    factory.setPoolName(POOL_NAME);
-    PoolFactoryImpl pf = (PoolFactoryImpl) PoolManager.createFactory();
-    pf.init(pool);
-    LocatorDiscoveryCallback locatorCallback = new MyLocatorCallback();
-    remoteObjects.put(CALLBACK_KEY, locatorCallback);
-    pf.setLocatorDiscoveryCallback(locatorCallback);
-    pf.create(POOL_NAME);
-
-    RegionAttributes attrs = factory.create();
-    for (int i = 0; i < regions.length; i++) {
-      cache.createRegion(regions[i], attrs);
+  
+  protected void  startBridgeClientInVM(VM vm, final Pool pool, final String[] regions) throws Exception {
+    SerializableRunnable connect =
+      new SerializableRunnable("Start bridge client") {
+          public void run() {
+            Properties props = new Properties();
+            props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+            props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+            DistributedSystem ds = getSystem(props);
+            Cache cache = CacheFactory.create(ds);
+            AttributesFactory factory = new AttributesFactory();
+            factory.setScope(Scope.LOCAL);
+//            factory.setEnableBridgeConflation(true);
+//            factory.setDataPolicy(DataPolicy.NORMAL);
+            factory.setPoolName(POOL_NAME);
+            PoolFactoryImpl pf= (PoolFactoryImpl) PoolManager.createFactory();
+            pf.init(pool);
+            LocatorDiscoveryCallback locatorCallback = new MyLocatorCallback();
+            remoteObjects.put(CALLBACK_KEY, locatorCallback);
+            pf.setLocatorDiscoveryCallback(locatorCallback);
+            pf.create(POOL_NAME);
+
+
+            RegionAttributes attrs = factory.create();
+            for(int i = 0; i < regions.length; i++) {
+              cache.createRegion(regions[i], attrs);
+            }
+            
+            remoteObjects.put(CACHE_KEY, cache);
+          }
+    };
+    
+    if(vm == null) {
+      connect.run();
+    } else {
+      vm.invoke(connect);
     }
-
-    remoteObjects.put(CACHE_KEY, cache);
   }
-
+  
   protected void stopBridgeMemberVM(VM vm) {
-    vm.invoke(new SerializableRunnable("Stop bridge member") {
-      public void run() {
-        Cache cache = (Cache) remoteObjects.remove(CACHE_KEY);
-        cache.close();
-        disconnectFromDS();
-      }
-    });
+   vm.invoke(new SerializableRunnable("Stop bridge member") {
+    public void run() {
+      Cache cache = (Cache) remoteObjects.remove(CACHE_KEY);
+      cache.close();
+      disconnectFromDS();
+    }
+   });
   }
-
+  
   public String getLocatorString(Host host, int locatorPort) {
-    return getLocatorString(host, new int[] { locatorPort });
+    return getLocatorString(host, new int[] {locatorPort});
   }
-
+  
   public String getLocatorString(Host host, int[] locatorPorts) {
     StringBuffer str = new StringBuffer();
-    for (int i = 0; i < locatorPorts.length; i++) {
+    for(int i = 0; i < locatorPorts.length; i++) {
       str.append(NetworkUtils.getServerHostName(host))
           .append("[")
           .append(locatorPorts[i])
           .append("]");
-      if (i < locatorPorts.length - 1) {
+      if(i < locatorPorts.length - 1) {
         str.append(",");
       }
     }
-
+    
     return str.toString();
   }
-
+  
   protected static class MyLocatorCallback extends LocatorDiscoveryCallbackAdapter {
 
     private final Set discoveredLocators = new HashSet();
     private final Set removedLocators = new HashSet();
-
+    
     public synchronized void locatorsDiscovered(List locators) {
       discoveredLocators.addAll(locators);
       notifyAll();
@@ -266,29 +336,29 @@ public abstract class LocatorTestBase extends DistributedTestCase {
       removedLocators.addAll(locators);
       notifyAll();
     }
-
+    
     public boolean waitForDiscovery(InetSocketAddress locator, long time) throws InterruptedException {
       return waitFor(discoveredLocators, locator, time);
     }
-
+    
     public boolean waitForRemove(InetSocketAddress locator, long time) throws InterruptedException {
       return waitFor(removedLocators, locator, time);
     }
-
+    
     private synchronized boolean waitFor(Set set, InetSocketAddress locator, long time) throws InterruptedException {
       long remaining = time;
       long endTime = System.currentTimeMillis() + time;
-      while (!set.contains(locator) && remaining >= 0) {
+      while(!set.contains(locator) && remaining >= 0) {
         wait(remaining);
-        remaining = endTime - System.currentTimeMillis();
+        remaining = endTime - System.currentTimeMillis(); 
       }
       return set.contains(locator);
     }
-
+    
     public synchronized Set getDiscovered() {
       return new HashSet(discoveredLocators);
     }
-
+    
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug47667DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug47667DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug47667DUnitTest.java
index 0f08456..de43c29 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug47667DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug47667DUnitTest.java
@@ -56,11 +56,11 @@ public class Bug47667DUnitTest extends LocatorTestBase {
 
     final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     final String locatorHost = NetworkUtils.getServerHostName(host);
-    locator.invoke("Start Locator",() ->startLocator(locator.getHost(), locatorPort, ""));
+    startLocatorInVM(locator, locatorPort, "");
 
     String locString = getLocatorString(host, locatorPort);
-    server1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] {"R1"}, locString, new String[] {"R1"}));
-    server2.invoke("Start BridgeServer", () -> startBridgeServer(new String[] {"R2"}, locString, new String[] {"R2"}));
+    startBridgeServerInVM(server1, new String[] {"R1"}, locString, new String[] {"R1"});
+    startBridgeServerInVM(server2, new String[] {"R2"}, locString, new String[] {"R2"});
 
     client.invoke(new SerializableCallable() {
       @Override
@@ -70,15 +70,15 @@ public class Bug47667DUnitTest extends LocatorTestBase {
         ClientCache cache = ccf.create();
         PoolManager.createFactory().addLocator(locatorHost, locatorPort).setServerGroup("R1").create("R1");
         PoolManager.createFactory().addLocator(locatorHost, locatorPort).setServerGroup("R2").create("R2");
-        Region region1 = cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).setPoolName("R1").create("R1");
-        Region region2 = cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).setPoolName("R2").create("R2");
-        CacheTransactionManager transactionManager = cache.getCacheTransactionManager();
-        transactionManager.begin();
-        region1.put(1, "value1");
-        transactionManager.commit();
-        transactionManager.begin();
-        region2.put(2, "value2");
-        transactionManager.commit();
+        Region r1 = cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).setPoolName("R1").create("R1");
+        Region r2 = cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).setPoolName("R2").create("R2");
+        CacheTransactionManager mgr = cache.getCacheTransactionManager();
+        mgr.begin();
+        r1.put(1, "value1");
+        mgr.commit();
+        mgr.begin();
+        r2.put(2, "value2");
+        mgr.commit();
         return null;
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
index 167cc3a..d784397 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/management/CacheServerManagementDUnitTest.java
@@ -16,9 +16,26 @@
  */
 package com.gemstone.gemfire.management;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Properties;
+
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanServer;
+import javax.management.Notification;
+import javax.management.NotificationListener;
+import javax.management.ObjectName;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.client.internal.LocatorTestBase;
-import com.gemstone.gemfire.cache.query.*;
+import com.gemstone.gemfire.cache.query.IndexExistsException;
+import com.gemstone.gemfire.cache.query.IndexInvalidException;
+import com.gemstone.gemfire.cache.query.IndexNameConflictException;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.RegionNotFoundException;
 import com.gemstone.gemfire.cache.query.cq.dunit.CqQueryDUnitTest;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.server.CacheServer;
@@ -29,34 +46,37 @@ import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.management.internal.JmxManagerLocatorRequest;
+import com.gemstone.gemfire.management.internal.JmxManagerLocatorResponse;
 import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
 import com.gemstone.gemfire.management.internal.SystemManagementService;
-import com.gemstone.gemfire.test.dunit.*;
-
-import javax.management.*;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Collections;
-import java.util.Properties;
+import com.gemstone.gemfire.test.dunit.Assert;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 
 /**
  * Cache Server related management test cases
+ * 
+ * 
  */
 public class CacheServerManagementDUnitTest extends LocatorTestBase {
 
   private static final long serialVersionUID = 1L;
-
-  private static int CONNECT_LOCATOR_TIMEOUT_MS = 30000;
+  
+  private static int CONNECT_LOCATOR_TIMEOUT_MS = 30000; 
 
   private ManagementTestBase helper;
 
   private static final String queryName = "testClientWithFeederAndCQ_0";
 
   private static final String indexName = "testIndex";
-
+  
   private static MBeanServer mbeanServer = MBeanJMXAdapter.mbeanServer;
+  
 
   protected CqQueryDUnitTest cqDUnitTest = new CqQueryDUnitTest(
       "CqDataDUnitTest");
@@ -64,7 +84,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
   public CacheServerManagementDUnitTest(String name) {
     super(name);
     this.helper = new ManagementTestBase(name);
-
+    
   }
 
   @Override
@@ -78,6 +98,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
   }
 
   /**
+   * 
    * @throws Exception
    */
   public void testCacheServerMBean() throws Exception {
@@ -91,11 +112,12 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     helper.startManagingNode(managingNode);
     //helper.createCache(server);
     int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    cqDUnitTest.createServer(server, serverPort);
-
+    cqDUnitTest.createServer(server,serverPort);
+    
+   
     DistributedMember member = helper.getMember(server);
-
-    verifyCacheServer(server, serverPort);
+    
+    verifyCacheServer(server,serverPort);
 
     final int port = server.invoke(() -> CqQueryDUnitTest.getCacheServerPort());
     final String host0 = NetworkUtils.getServerHostName(server.getHost());
@@ -123,10 +145,10 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     // Close.
 
     Wait.pause(2000);
-    checkNavigation(managingNode, member, serverPort);
-    verifyIndex(server, serverPort);
+    checkNavigation(managingNode,member,serverPort);
+    verifyIndex(server,serverPort);
     // This will test all CQs and will close the cq in its final step
-    verifyCacheServerRemote(managingNode, member, serverPort);
+    verifyCacheServerRemote(managingNode, member,serverPort);
 
     verifyClosedCQ(server);
 
@@ -140,30 +162,30 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
 
   /**
    * Test for client server connection related management artifacts
-   * like notifications
-   *
+   * like notifications 
    * @throws Exception
    */
-
+  
   public void testCacheClient() throws Exception {
-
+    
     final Host host = Host.getHost(0);
     VM locator = host.getVM(0);
     VM server = host.getVM(1);
     VM client = host.getVM(2);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    locator.invoke("Start Locator", () -> startLocator(locator.getHost(), locatorPort, ""));
-
-    String locators = NetworkUtils.getServerHostName(locator.getHost()) + "[" + locatorPort + "]";
-
-    int serverPort = server.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
-    addClientNotifListener(server, serverPort);
+    startLocatorInVM(locator, locatorPort, "");
+    
+    String locators = NetworkUtils.getServerHostName(locator.getHost())+ "[" + locatorPort + "]";
+    
+   
+    int serverPort = startBridgeServerInVM(server, null, locators);
+    
+    addClientNotifListener(server,serverPort);
 
     // Start a client and make sure that proper notification is received
-    client.invoke("Start BridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(locator.getHost()), locatorPort));
-
+    startBridgeClientInVM(client, null, NetworkUtils.getServerHostName(locator.getHost()), locatorPort);
+    
     //stop the client and make sure the bridge server notifies
     stopBridgeMemberVM(client);
     helper.closeCache(locator);
@@ -171,14 +193,13 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     helper.closeCache(client);
 
   }
-
+  
   /**
    * Intention of this test is to check if a node becomes manager after all the nodes are alive
    * it should have all the information of all  the members.
-   * <p>
+   * 
    * Thats why used  service.getLocalManager().runManagementTaskAdhoc() to make node
    * ready for federation when manager node comes up
-   *
    * @throws Exception
    */
 
@@ -187,72 +208,96 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     final Host host = Host.getHost(0);
     VM locator = host.getVM(0);
     VM server = host.getVM(1);
-
+    
     //Step 1:
     final int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    locator.invoke("Start Locator", () -> startLocator(locator.getHost(), locatorPort, ""));
-
-    String locators = NetworkUtils.getServerHostName(locator.getHost()) + "[" + locatorPort + "]";
+    startLocator(locator, locatorPort, "");
 
+    String locators = NetworkUtils.getServerHostName(locator.getHost())+ "[" + locatorPort + "]";
+    
     //Step 2:
-    server.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
+    int serverPort = startBridgeServerInVM(server, null, locators);
+    
     //Step 3:
-    server.invoke("Check Server", () -> {
-      Cache cache = GemFireCacheImpl.getInstance();
-      assertNotNull(cache);
-      SystemManagementService service = (SystemManagementService) ManagementService
-          .getExistingManagementService(cache);
-      assertNotNull(service);
-      assertFalse(service.isManager());
-      assertNotNull(service.getMemberMXBean());
-      service.getLocalManager().runManagementTaskAdhoc();
+    server.invoke(new SerializableRunnable("Check Server") {
+
+      public void run() {
+        Cache cache = GemFireCacheImpl.getInstance();
+        assertNotNull(cache);
+        SystemManagementService service = (SystemManagementService)ManagementService
+            .getExistingManagementService(cache);
+        assertNotNull(service);
+        assertFalse(service.isManager());
+        assertNotNull(service.getMemberMXBean());
+        service.getLocalManager().runManagementTaskAdhoc();
+
+
+      }
     });
+    
+  //Step 4:
+    JmxManagerLocatorResponse locRes = JmxManagerLocatorRequest.send(locator
+        .getHost().getHostName(), locatorPort, CONNECT_LOCATOR_TIMEOUT_MS, Collections.<String, String> emptyMap());
+    
+  //Step 5:
+    locator.invoke(new SerializableRunnable("Check locator") {
 
-    //Step 4:
-    JmxManagerLocatorRequest.send(locator
-        .getHost().getHostName(), locatorPort, CONNECT_LOCATOR_TIMEOUT_MS, Collections.<String, String>emptyMap());
-
-    //Step 5:
-    locator.invoke("Check locator", () -> {
-      Cache cache = GemFireCacheImpl.getInstance();
-      assertNotNull(cache);
-      ManagementService service = ManagementService
-          .getExistingManagementService(cache);
-      assertNotNull(service);
-      assertTrue(service.isManager());
-      LocatorMXBean bean = service.getLocalLocatorMXBean();
-      assertEquals(locatorPort, bean.getPort());
-      DistributedSystemMXBean dsBean = service.getDistributedSystemMXBean();
-
-      assertEquals(2, dsBean.listMemberObjectNames().length);
+      public void run() {
+        Cache cache = GemFireCacheImpl.getInstance();
+        assertNotNull(cache);
+        ManagementService service = ManagementService
+            .getExistingManagementService(cache);
+        assertNotNull(service);
+        assertTrue(service.isManager());
+        LocatorMXBean bean = service.getLocalLocatorMXBean();
+        assertEquals(locatorPort, bean.getPort());
+        DistributedSystemMXBean dsBean = service.getDistributedSystemMXBean();
+        ObjectName[] names = dsBean.listMemberObjectNames();
+       
+        assertEquals(2,dsBean.listMemberObjectNames().length);
+
+      }
     });
+    
+
 
     helper.closeCache(locator);
     helper.closeCache(server);
-
+    
+ 
   }
+  
+  
+  protected void startLocator(final VM vm, final int locatorPort, final String otherLocators) {
+    vm.invoke(new SerializableRunnable("Create Locator") {
 
-  protected void startLocator(Host vmHost, final int locatorPort, final String otherLocators) {
-    disconnectFromDS();
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
-    props.setProperty(DistributionConfig.LOCATORS_NAME, otherLocators);
-    props.setProperty(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
-    props.setProperty(DistributionConfig.JMX_MANAGER_HTTP_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
-    File logFile = new File(getUniqueName() + "-locator" + locatorPort + ".log");
-    try {
-      InetAddress bindAddr = InetAddress.getByName(NetworkUtils.getServerHostName(vmHost));
-      Locator locator = Locator.startLocatorAndDS(locatorPort, logFile, bindAddr, props);
-      remoteObjects.put(LOCATOR_KEY, locator);
-    } catch (UnknownHostException uhe) {
-      Assert.fail("While resolving bind address ", uhe);
-    } catch (IOException ex) {
-      Assert.fail("While starting locator on port " + locatorPort, ex);
-    }
+      final String testName= getUniqueName();
+      public void run() {
+        disconnectFromDS();
+        Properties props = new Properties();
+        props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+        props.setProperty(DistributionConfig.LOCATORS_NAME, otherLocators);
+        props.setProperty(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
+        props.setProperty(DistributionConfig.JMX_MANAGER_HTTP_PORT_NAME, "0");
+        props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
+        try {
+          File logFile = new File(testName + "-locator" + locatorPort
+              + ".log");
+          InetAddress bindAddr = null;
+          try {
+            bindAddr = InetAddress.getByName(NetworkUtils.getServerHostName(vm.getHost()));
+          } catch (UnknownHostException uhe) {
+            Assert.fail("While resolving bind address ", uhe);
+          }
+          Locator locator = Locator.startLocatorAndDS(locatorPort, logFile, bindAddr, props);
+          remoteObjects.put(LOCATOR_KEY, locator);
+        } catch (IOException ex) {
+          Assert.fail("While starting locator on port " + locatorPort, ex);
+        }
+      }
+    });
   }
-
+  
   protected void checkNavigation(final VM vm,
       final DistributedMember cacheServerMember, final int serverPort) {
     SerializableRunnable checkNavigation = new SerializableRunnable(
@@ -282,14 +327,14 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     };
     vm.invoke(checkNavigation);
   }
-
+  
   /**
    * Verify the Cache Server details
-   *
+   * 
    * @param vm
    */
   @SuppressWarnings("serial")
-  protected void addClientNotifListener(final VM vm, final int serverPort) throws Exception {
+  protected void addClientNotifListener(final VM vm , final int serverPort) throws Exception {
     SerializableRunnable addClientNotifListener = new SerializableRunnable(
         "Add Client Notif Listener") {
       public void run() {
@@ -314,19 +359,18 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
         TestCacheServerNotif nt = new TestCacheServerNotif();
         try {
           mbeanServer.addNotificationListener(MBeanJMXAdapter
-              .getClientServiceMBeanName(serverPort, cache.getDistributedSystem().getMemberId()), nt, null, null);
+              .getClientServiceMBeanName(serverPort,cache.getDistributedSystem().getMemberId()), nt, null, null);
         } catch (InstanceNotFoundException e) {
           fail("Failed With Exception " + e);
         }
-
+        
       }
     };
     vm.invoke(addClientNotifListener);
   }
-
   /**
    * Verify the closed CQ which is closed from Managing Node
-   *
+   * 
    * @param vm
    */
   @SuppressWarnings("serial")
@@ -361,7 +405,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
           bean.removeIndex(indexName);
         } catch (Exception e) {
           fail("Failed With Exception " + e);
-
+        
         }
         assertEquals(bean.getIndexCount(), 0);
 
@@ -372,7 +416,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
 
   /**
    * Verify the closed CQ which is closed from Managing Node
-   *
+   * 
    * @param vm
    */
   @SuppressWarnings("serial")
@@ -390,9 +434,11 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     vm.invoke(verifyClosedCQ);
   }
 
+ 
+
   /**
    * Verify the Cache Server details
-   *
+   * 
    * @param vm
    */
   @SuppressWarnings("serial")
@@ -450,7 +496,7 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
 
   /**
    * Verify the Cache Server details
-   *
+   * 
    * @param vm
    */
   @SuppressWarnings("serial")
@@ -476,14 +522,14 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
           LogWriterUtils.getLogWriter().info(
               "<ExpectedString> Active Query Count  "
                   + bean.getActiveCQCount() + "</ExpectedString> ");
-
+          
           LogWriterUtils.getLogWriter().info(
               "<ExpectedString> Registered Query Count  "
                   + bean.getRegisteredCQCount() + "</ExpectedString> ");
 
-          assertTrue(bean.showAllClientStats()[0].getClientCQCount() == 1);
-          int numQueues = bean.getNumSubscriptions();
-          assertEquals(numQueues, 1);
+          assertTrue(bean.showAllClientStats()[0].getClientCQCount() == 1); 
+          int numQueues = bean.getNumSubscriptions(); 
+          assertEquals(numQueues, 1); 
           // test for client connection Count
           
           /* @TODO */
@@ -506,9 +552,11 @@ public class CacheServerManagementDUnitTest extends LocatorTestBase {
     };
     vm.invoke(verifyCacheServerRemote);
   }
-
+  
   /**
    * Notification handler
+   * 
+   * 
    */
   private static class TestCacheServerNotif implements
       NotificationListener {


[09/50] [abbrv] incubator-geode git commit: GEODE-430: Fixing races in UpdatePropagationDUnitTest

Posted by kl...@apache.org.
GEODE-430: Fixing races in UpdatePropagationDUnitTest

This test was failing with a couple of different race conditions

1) It was not waiting for updates to make it to the second client,
causing assertion failures. I added a wait.

2) It was occasionally using a connection that was previously made to
the server before it was killed, causing an unexpected socket closed. I
refactored the test to not use internal APIs to do the put, but instead
verify that the event was not sent to the client using a listener.

3) In the PR version of the test, the PR single hop code can return
different addresses for the server, resulting in the pool thinking it
has two different servers when it only has one. I changed the wait to
wait for a server with a given port to go away.

I also did some cleanup of the test - extended JUnit4CacheTestCase,
removed the waits, used awaitility, removed an almost duplicate disabled
test method and added the one extra assertion from that duplicate test
to the enabled test method.


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

Branch: refs/heads/feature/GEODE-835
Commit: 884cf13b0ccace80d15ca11bcc3162f5169bdf73
Parents: c79b64f
Author: Dan Smith <up...@apache.org>
Authored: Wed May 11 13:04:57 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri May 13 15:41:56 2016 -0700

----------------------------------------------------------------------
 .../sockets/UpdatePropagationDUnitTest.java     | 489 +++++--------------
 .../sockets/UpdatePropagationPRDUnitTest.java   |   4 +-
 2 files changed, 136 insertions(+), 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/884cf13b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
index 69781c4..08d964a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
@@ -16,44 +16,50 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
+import static junit.framework.TestCase.assertNotNull;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
-import org.junit.Ignore;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.CacheWriterException;
 import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.EntryEvent;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.EndpointManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
 import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
-import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
-import com.gemstone.gemfire.test.dunit.WaitCriterion;
-import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.jayway.awaitility.Awaitility;
+import com.jayway.awaitility.core.ConditionTimeoutException;
 
 /**
  * Start client 1
@@ -71,7 +77,8 @@ import com.gemstone.gemfire.test.junit.categories.FlakyTest;
  * The key is to verify that the memberid being used by the client
  * to register with the server is the same across servers
  */
-public class UpdatePropagationDUnitTest extends DistributedTestCase {
+@Category(DistributedTest.class)
+public class UpdatePropagationDUnitTest extends JUnit4CacheTestCase {
 
   VM server1 = null;
 
@@ -87,13 +94,6 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
 
   private static final String REGION_NAME = "UpdatePropagationDUnitTest_region";
 
-  protected static Cache cache = null;
-  
-  /** constructor */
-  public UpdatePropagationDUnitTest(String name) {
-    super(name);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     disconnectAllFromDS();
@@ -111,382 +111,169 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
     //client 2 VM
     client2 = host.getVM(3);
     
-    PORT1 =  ((Integer)server1.invoke(() -> createServerCache())).intValue();
-    PORT2 =  ((Integer)server2.invoke(() -> createServerCache())).intValue();
+    PORT1 =  server1.invoke(() -> createServerCache());
+    PORT2 =  server2.invoke(() -> createServerCache());
 
     client1.invoke(() -> createClientCache(
-      NetworkUtils.getServerHostName(server1.getHost()), new Integer(PORT1),new Integer(PORT2)));
+      NetworkUtils.getServerHostName(server1.getHost()), PORT1, PORT2));
     client2.invoke(() -> createClientCache(
-      NetworkUtils.getServerHostName(server1.getHost()), new Integer(PORT1),new Integer(PORT2)));
+      NetworkUtils.getServerHostName(server1.getHost()), PORT1, PORT2));
     
     IgnoredException.addIgnoredException("java.net.SocketException");
     IgnoredException.addIgnoredException("Unexpected IOException");
   }
 
-  private void createCache(Properties props) throws Exception
-  {
-    DistributedSystem ds = getSystem(props);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  static private final String WAIT_PROPERTY =
-    "UpdatePropagationDUnitTest.maxWaitTime";
-  static private final int WAIT_DEFAULT = 60000;
-
-
-  /**
-   * This tests whether the updates are received by the sender or not if
-   * there is an Interest List failover
-   *
-   */
-  @Ignore("Bug 50405")
-  public void DISABLED_testVerifyUpdatesNotReceivedBySender()
-  {
-    final int maxWaitTime = Integer.getInteger(WAIT_PROPERTY, WAIT_DEFAULT).intValue();
-
-    //First create entries on both servers via the two client
-    client1.invoke(() -> createEntriesK1andK2());
-    client2.invoke(() -> createEntriesK1andK2());
-    client1.invoke(() -> registerKeysK1andK2());
-    client2.invoke(() -> registerKeysK1andK2());
-    //Induce fail over of InteretsList Endpoint to Server 2 by killing server1
-    
-    server1.invoke(() -> UpdatePropagationDUnitTest.killServer(new Integer(PORT1)));
-    //Wait for 10 seconds to allow fail over. This would mean that Interest
-    // has failed over to Server2.
-    client1.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong1"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-
-    client2.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong3"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-
-    //Start Server1 again so that both clients1 & Client 2 will establish connection to server1 too.
-    server1.invoke(() -> UpdatePropagationDUnitTest.startServer(new Integer(PORT1)));
-
-    client1.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong2"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() == 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-
-    //Do a put on Server1 via Connection object from client1.
-    // Client1 should not receive updated value while client2 should receive
-    client1.invoke(() -> acquireConnectionsAndPutonK1andK2( NetworkUtils.getServerHostName(client1.getHost())));
-    //pause(5000);
-    //Check if both the puts ( on key1 & key2 ) have reached the servers
-    server1.invoke(() -> verifyUpdates());
-    server2.invoke(() -> verifyUpdates());
-    // verify no updates for update originator
-    client1.invoke(() -> verifyNoUpdates());
-
-  }
-
-
   /**
    * This tests whether the updates are received by other clients or not , if there are
    * situation of Interest List fail over
    */
-  @Category(FlakyTest.class) // GEODE-430: time sensitive, random ports, port reuse, thread sleeps (5 seconds), eats exceptions (fixed 1), async actions, waitForCriterion
-  public void testVerifyUpdatesReceivedByOtherClients() {
-    final int maxWaitTime = Integer.getInteger(WAIT_PROPERTY, WAIT_DEFAULT).intValue();
+  @Test
+  public void updatesAreProgegatedAfterFailover() {
     //  First create entries on both servers via the two client
     client1.invoke(() -> createEntriesK1andK2());
     client2.invoke(() -> createEntriesK1andK2());
     client1.invoke(() -> registerKeysK1andK2());
     client2.invoke(() -> registerKeysK1andK2());
     //Induce fail over of InteretsList Endpoint to Server 2 by killing server1
-    server1.invoke(() -> UpdatePropagationDUnitTest.killServer(new Integer(PORT1)));
+    server1.invoke(() -> killServer(new Integer(PORT1)));
     //Wait for 10 seconds to allow fail over. This would mean that Interstist has failed
     // over to Server2.
-    client1.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong3"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
-
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
-    client2.invoke(new CacheSerializableRunnable("Wait for server on port1 to be dead") {
+    final CacheSerializableRunnable waitToDetectDeadServer = new CacheSerializableRunnable("Wait for server on port1 to be dead") {
       public void run2() throws CacheException
       {
-        Region r = cache.getRegion(REGION_NAME);
-
-        try {
-          r.put("ping", "pong3"); // in the event there is no live server monitor thread
-        } catch (CacheWriterException itsOk) {}
+        Region r = getCache().getRegion(REGION_NAME);
 
         String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() != 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
+        final PoolImpl pool = (PoolImpl) PoolManager.find(poolName);
+        Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> !hasEndPointWithPort(pool, PORT1));
       }
-    });
+    };
+    client1.invoke(waitToDetectDeadServer);
+    client2.invoke(waitToDetectDeadServer);
 
     //Start Server1 again so that both clients1 & Client 2 will establish connection to server1 too.
-    server1.invoke(() -> UpdatePropagationDUnitTest.startServer(new Integer(PORT1)));
-
-    client1.invoke(new CacheSerializableRunnable("Wait for servers to be alive") {
-      public void run2() throws CacheException
-      {
-        Region r = cache.getRegion(REGION_NAME);
-        String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() == 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
-      }
-    });
+    server1.invoke(() -> startServer(new Integer(PORT1)));
 
-    client2.invoke(new CacheSerializableRunnable("Wait for servers to be alive") {
+    final CacheSerializableRunnable waitToDetectLiveServer = new CacheSerializableRunnable("Wait for servers to be alive") {
       public void run2() throws CacheException
       {
-        Region r = cache.getRegion(REGION_NAME);
+        Region r = getCache().getRegion(REGION_NAME);
         String poolName = r.getAttributes().getPoolName();
-        assertNotNull(poolName);
-        final PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-        assertNotNull(pool);
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return pool.getConnectedServerCount() == 2;
-          }
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, maxWaitTime, 200, true);
+        final PoolImpl pool = (PoolImpl) PoolManager.find(poolName);
+        Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> hasEndPointWithPort(pool, PORT1));
       }
-    });
-    
-    Wait.pause(5000);
+    };
+    client1.invoke(waitToDetectLiveServer);
+    client2.invoke(waitToDetectLiveServer);
 
     //Do a put on Server1 via Connection object from client1.
     // Client1 should not receive updated value while client2 should receive
     client1.invoke(() -> acquireConnectionsAndPutonK1andK2( NetworkUtils.getServerHostName(client1.getHost())));
-    Wait.pause(5000);
     //Check if both the puts ( on key1 & key2 ) have reached the servers
     server1.invoke(() -> verifyUpdates());
     server2.invoke(() -> verifyUpdates());
     // verify updates to other client
     client2.invoke(() -> verifyUpdates());
+
+    // verify no updates for update originator
+    client1.invoke(() -> verifySenderUpdateCount());
+  }
+
+  /**
+   * Check to see if a client is connected to an endpoint with a specific port
+   */
+  private boolean hasEndPointWithPort(final PoolImpl pool, final int port) {
+    EndpointManager endpointManager = pool.getEndpointManager();
+    final Set<ServerLocation> servers = endpointManager
+      .getEndpointMap().keySet();
+    return servers.stream().anyMatch(location -> location.getPort() == port);
   }
 
   public void acquireConnectionsAndPutonK1andK2(String host)
   {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r1);
-      String poolName = r1.getAttributes().getPoolName();
-      assertNotNull(poolName);
-      PoolImpl pool = (PoolImpl)PoolManager.find(poolName);
-      assertNotNull(pool);
-
-      Connection conn = pool.acquireConnection(new ServerLocation(host,PORT1));
-      assertNotNull(conn);
-      assertEquals(PORT1, conn.getServer().getPort());
-      ServerRegionProxy srp = new ServerRegionProxy(Region.SEPARATOR+ REGION_NAME, pool);
-      srp.putOnForTestsOnly(conn, "key1", "server-value1", new EventID(new byte[] {1},159632,1), null);
-      srp.putOnForTestsOnly(conn, "key2", "server-value2", new EventID(new byte[] {1},159632,2), null);
-    }
-    catch (Exception ex) {
-      Assert.fail("while setting acquireConnections", ex);
-    }
+    Region r1 = getCache().getRegion(Region.SEPARATOR + REGION_NAME);
+    r1.put("key1", "server-value1");
+    r1.put("key2", "server-value2");
   }
 
-  public static void killServer(Integer port )
+  public void killServer(Integer port )
   {
-    try {
-      Iterator iter = cache.getCacheServers().iterator();
-      if (iter.hasNext()) {
-        CacheServer server = (CacheServer)iter.next();
-        if(server.getPort() == port.intValue()){
-          server.stop();
-        }
-
+    Iterator iter = getCache().getCacheServers().iterator();
+    if (iter.hasNext()) {
+      CacheServer server = (CacheServer)iter.next();
+      if(server.getPort() == port.intValue()){
+        server.stop();
       }
-    }
-    catch (Exception ex) {
-      fail("while killing Server  " + ex);
+
     }
   }
 
-  public static void startServer(Integer port)
+  public void startServer(Integer port) throws IOException
   {
-    try {
-      CacheServer server1 = cache.addCacheServer();
-      server1.setPort(port.intValue());
-      server1.setNotifyBySubscription(true);
-      server1.start();
-    }
-    catch (Exception ex) {
-      fail("while killServer  " + ex);
-    }
+    CacheServer server1 = getCache().addCacheServer();
+    server1.setPort(port.intValue());
+    server1.setNotifyBySubscription(true);
+    server1.start();
   }
 
   /**
    * Creates entries on the server
    *
    */
-  public static void createEntriesK1andK2()
+  public void createEntriesK1andK2()
   {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR+REGION_NAME);
-      assertNotNull(r1);
-      if (!r1.containsKey("key1")) {
-        r1.put("key1", "key-1");
-      }
-      if (!r1.containsKey("key2")) {
-        r1.put("key2", "key-2");
-      }
-      assertEquals(r1.get("key1"), "key-1");
-      if (r1.getAttributes().getPartitionAttributes() == null) {
-        assertEquals(r1.getEntry("key1").getValue(), "key-1");
-        assertEquals(r1.getEntry("key2").getValue(), "key-2");
-      }
-      else {
-        assertEquals(r1.get("key1"), "key-1");
-        assertEquals(r1.get("key2"), "key-2");
-      }
+    Region r1 = getCache().getRegion(Region.SEPARATOR+REGION_NAME);
+    assertNotNull(r1);
+    if (!r1.containsKey("key1")) {
+      r1.put("key1", "key-1");
     }
-    catch (Exception ex) {
-      Assert.fail("failed while createEntriesK1andK2()", ex);
+    if (!r1.containsKey("key2")) {
+      r1.put("key2", "key-2");
+    }
+    assertEquals(r1.get("key1"), "key-1");
+    if (r1.getAttributes().getPartitionAttributes() == null) {
+      assertEquals(r1.getEntry("key1").getValue(), "key-1");
+      assertEquals(r1.getEntry("key2").getValue(), "key-2");
+    }
+    else {
+      assertEquals(r1.get("key1"), "key-1");
+      assertEquals(r1.get("key2"), "key-2");
     }
   }
 
-  public static void createClientCache(String host, Integer port1 , Integer port2 ) throws Exception
+  public void createClientCache(String host, Integer port1 , Integer port2 ) throws Exception
   {
-    int PORT1 = port1.intValue() ;
-    int PORT2 = port2.intValue();
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new UpdatePropagationDUnitTest("temp").createCache(props);
-    Pool p;
+    ClientCache cache;
     try {
       System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "true");
-      p = PoolManager.createFactory()
-        .addServer(host, PORT1)
-        .addServer(host, PORT2)
-        .setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(-1)
-        .setMinConnections(4)
-        .setSocketBufferSize(1000)
-        .setReadTimeout(2000)
-        // .setRetryInterval(250)
-        // .setRetryAttempts(2)
-        .create("UpdatePropagationDUnitTestPool");
+      int PORT1 = port1.intValue() ;
+      int PORT2 = port2.intValue();
+      Properties props = new Properties();
+      props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+      props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+      ClientCacheFactory cf = new ClientCacheFactory();
+      cf.addPoolServer(host, PORT1)
+      .addPoolServer(host, PORT2)
+      .setPoolSubscriptionEnabled(true)
+      .setPoolSubscriptionRedundancy(-1)
+      .setPoolMinConnections(4)
+      .setPoolSocketBufferSize(1000)
+      .setPoolReadTimeout(2000)
+      .setPoolPingInterval(300);
+       cache = getClientCache(cf);
     } finally {
       System.setProperty("gemfire.PoolImpl.DISABLE_RANDOM", "false");
       CacheServerTestUtil.enableShufflingOfEndpoints();
     }
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-
+    cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
+      .addCacheListener(new EventTrackingCacheListener())
+      .create(REGION_NAME);
   }
 
   public Integer createServerCache() throws Exception
   {
-    new UpdatePropagationDUnitTest("temp").createCache(new Properties());
-    RegionAttributes attrs = createCacheServerAttributes(); 
+    Cache cache = getCache();
+    RegionAttributes attrs = createCacheServerAttributes();
     cache.createRegion(REGION_NAME, attrs);
     CacheServer server = cache.addCacheServer();
     assertNotNull(server);
@@ -505,10 +292,10 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
     return factory.create();
   }
 
-  public static void registerKeysK1andK2()
+  public void registerKeysK1andK2()
   {
     try {
-      Region r = cache.getRegion(Region.SEPARATOR+ REGION_NAME);
+      Region r = getCache().getRegion(Region.SEPARATOR+ REGION_NAME);
       assertNotNull(r);
       List list = new ArrayList();
       list.add("key1");
@@ -521,25 +308,22 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
     }
   }
 
-  public static void verifyNoUpdates()
+  public void verifySenderUpdateCount()
   {
-    try {
-      Region r = cache.getRegion(Region.SEPARATOR+ REGION_NAME);
-      assertNotNull(r);
-      // verify no updates
-      assertEquals("key-1", r.getEntry("key1").getValue());
-      assertEquals("key-2", r.getEntry("key2").getValue());
-    }
-    catch (Exception ex) {
-      Assert.fail("failed while verifyNoUpdates()", ex);
-    }
+    Region r = getCache().getRegion(Region.SEPARATOR+ REGION_NAME);
+    EventTrackingCacheListener listener = (EventTrackingCacheListener) r.getAttributes().getCacheListeners()[0];
+
+    final List<EntryEvent> events = listener.receivedEvents;
+
+    //We only expect to see 1 create and 1 update from the original put
+    assertEquals("Expected only 2 events for key1", 2, events.stream().filter(event -> event.getKey().equals("key1")).count());
+    assertEquals("Expected only 2 events for key2", 2, events.stream().filter(event -> event.getKey().equals("key2")).count());
   }
 
-  public static void verifyUpdates()
+  public void verifyUpdates()
   {
-    try {
-      Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r);
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      Region r = getCache().getRegion(Region.SEPARATOR + REGION_NAME);
       // verify updates
       if (r.getAttributes().getPartitionAttributes() == null) {
         assertEquals("server-value2", r.getEntry("key2").getValue());
@@ -549,28 +333,25 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase {
         assertEquals("server-value2", r.get("key2"));
         assertEquals("server-value1", r.get("key1"));
       }
+    });
+  }
+
+  private static class EventTrackingCacheListener extends CacheListenerAdapter {
+    List<EntryEvent> receivedEvents = new ArrayList<>();
+
+    @Override public void afterCreate(final EntryEvent event) {
+      receivedEvents.add(event);
     }
-    catch (Exception ex) {
-      Assert.fail("failed while region", ex);
+
+    @Override public void afterUpdate(final EntryEvent event) {
+      receivedEvents.add(event);
     }
-  }
 
-  public static void closeCache()
-  {
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
+    @Override public void afterDestroy(final EntryEvent event) {
+      receivedEvents.add(event);
     }
-  }
 
-  @Override
-  public final void preTearDown() throws Exception {
-    //close client
-    client1.invoke(() -> closeCache());
-    client2.invoke(() -> closeCache());
-    //close server
-    server1.invoke(() -> closeCache());
-    server2.invoke(() -> closeCache());
+
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/884cf13b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
index 8edac45..589b455 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationPRDUnitTest.java
@@ -24,9 +24,7 @@ import com.gemstone.gemfire.cache.*;
  */
 public class UpdatePropagationPRDUnitTest extends UpdatePropagationDUnitTest {
 
-  public UpdatePropagationPRDUnitTest(String name) {
-    super(name);
-  }
+  @Override
   protected RegionAttributes createCacheServerAttributes()
   {
     AttributesFactory factory = new AttributesFactory();


[19/50] [abbrv] incubator-geode git commit: GEODE-1402: use preTearDownAssertions to call checkOrphans

Posted by kl...@apache.org.
GEODE-1402: use preTearDownAssertions to call checkOrphans


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

Branch: refs/heads/feature/GEODE-835
Commit: 11da98d4a7139b7c049bc012b7a3f2caaede184a
Parents: 84a9eb1
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon May 16 17:37:27 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue May 17 09:55:03 2016 -0700

----------------------------------------------------------------------
 .../cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java   | 2 +-
 .../cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java | 2 +-
 .../gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java   | 2 +-
 .../gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java      | 2 +-
 .../gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java | 2 +-
 .../gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java    | 2 +-
 .../gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java  | 2 +-
 .../com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java | 2 +-
 .../gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java     | 2 +-
 .../gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java         | 2 +-
 .../internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java    | 2 +-
 .../gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java  | 2 +-
 .../gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java      | 2 +-
 .../cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java     | 2 +-
 .../internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java  | 2 +-
 .../tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java      | 2 +-
 .../compression/CompressionCacheListenerOffHeapDUnitTest.java      | 2 +-
 .../compression/CompressionRegionOperationsOffHeapDUnitTest.java   | 2 +-
 .../gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java      | 2 +-
 19 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
index 1f0fbd9..908bc96 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckOverflowRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckOverflowRegionCCEOffHeapDUnitTest extends Distributed
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
index 431116a..c1bd906 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckPersistentRegionCCEOffHeapDUnitTest extends Distribut
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
index 3950dca..ffba1d3 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckRegionCCEOffHeapDUnitTest extends DistributedAckRegio
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
index 1b2c6be..08486d4 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedAckRegionOffHeapDUnitTest extends DistributedAckRegionDU
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
index 98e92c0..08a1fcb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedNoAckRegionCCEOffHeapDUnitTest extends DistributedNoAckR
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
index 8778069..60b0fdf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class DistributedNoAckRegionOffHeapDUnitTest extends DistributedNoAckRegi
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
index 4311196..6846aa8 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionCCEOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class GlobalRegionCCEOffHeapDUnitTest extends GlobalRegionCCEDUnitTest {
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
index dda4f7c..cb0e7ec 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/GlobalRegionOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class GlobalRegionOffHeapDUnitTest extends GlobalRegionDUnitTest {
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
index 8af7b2a..af25980 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/OffHeapLRUEvictionControllerDUnitTest.java
@@ -40,7 +40,7 @@ public class OffHeapLRUEvictionControllerDUnitTest extends
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
index b00d9c0..897e9b9 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionOffHeapDUnitTest.java
@@ -37,7 +37,7 @@ public class PartitionedRegionOffHeapDUnitTest extends PartitionedRegionDUnitTes
   }
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
index b9c90dc..33513b9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/DistrbutedRegionProfileOffHeapDUnitTest.java
@@ -40,7 +40,7 @@ public class DistrbutedRegionProfileOffHeapDUnitTest extends CacheTestCase {
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
index 97477fe..d93aa89 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionDUnitTest.java
@@ -42,7 +42,7 @@ public class OffHeapEvictionDUnitTest extends EvictionDUnitTest {
   }  
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
index 1f410db..6920e05 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/OffHeapEvictionStatsDUnitTest.java
@@ -38,7 +38,7 @@ public class OffHeapEvictionStatsDUnitTest extends EvictionStatsDUnitTest {
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
index b204453..55e52cb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionLocalMaxMemoryOffHeapDUnitTest.java
@@ -38,7 +38,7 @@ public class PartitionedRegionLocalMaxMemoryOffHeapDUnitTest extends Partitioned
   }
   
   @Override
-  protected final void preTearDownPartitionedRegionDUnitTest() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
index 046d787..48f88e8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionOffHeapEvictionDUnitTest.java
@@ -34,7 +34,7 @@ public class PartitionedRegionOffHeapEvictionDUnitTest extends
   }  
   
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
index c62c593..c4fdded 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateOffHeapEvictionDUnitTest.java
@@ -35,7 +35,7 @@ public class ForceInvalidateOffHeapEvictionDUnitTest extends
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
index ec828fd..0100a05 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionCacheListenerOffHeapDUnitTest.java
@@ -40,7 +40,7 @@ public class CompressionCacheListenerOffHeapDUnitTest extends
   }
 
   @Override
-  protected final void preTearDownCompressionCacheListenerDUnitTest() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
index d76976f..0cc3bdf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/compression/CompressionRegionOperationsOffHeapDUnitTest.java
@@ -33,7 +33,7 @@ public class CompressionRegionOperationsOffHeapDUnitTest extends
   }
   
   @Override
-  protected final void preTearDownCompressionRegionOperationsDUnitTest() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     SerializableRunnable checkOrphans = new SerializableRunnable() {
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/11da98d4/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
index a1ce952..99fadfc 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/OutOfOffHeapMemoryDUnitTest.java
@@ -77,7 +77,7 @@ public class OutOfOffHeapMemoryDUnitTest extends CacheTestCase {
   }
 
   @Override
-  public final void preTearDownCacheTestCase() throws Exception {
+  public final void preTearDownAssertions() throws Exception {
     final SerializableRunnable checkOrphans = new SerializableRunnable() {
       @Override
       public void run() {


[14/50] [abbrv] incubator-geode git commit: GEODE-17: wording change

Posted by kl...@apache.org.
GEODE-17: wording change


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

Branch: refs/heads/feature/GEODE-835
Commit: b6de5a39ad78d74ef727bea5be76b6a12b69f237
Parents: 4428954
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon May 16 11:19:03 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon May 16 11:27:39 2016 -0700

----------------------------------------------------------------------
 .../gemfire/management/internal/cli/commands/ShellCommands.java  | 4 ++--
 .../gemfire/management/internal/cli/i18n/CliStrings.java         | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b6de5a39/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
index a9712a1..baf3049 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
@@ -200,7 +200,7 @@ public class ShellCommands implements CommandMarker {
 
         if (userName != null && userName.length() > 0) {
           if (passwordToUse == null || passwordToUse.length() == 0) {
-            passwordToUse = gemfireShell.readWithMask("http password: ", '*');
+            passwordToUse = gemfireShell.readWithMask("password: ", '*');
           }
           if (passwordToUse == null || passwordToUse.length() == 0) {
             throw new IllegalArgumentException(CliStrings.CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED);
@@ -266,7 +266,7 @@ public class ShellCommands implements CommandMarker {
         // JMX Authentication Config
         if (userName != null && userName.length() > 0) {
           if (passwordToUse == null || passwordToUse.length() == 0) {
-            passwordToUse = gfshInstance.readWithMask("jmx password: ", '*');
+            passwordToUse = gfshInstance.readWithMask("password: ", '*');
           }
           if (passwordToUse == null || passwordToUse.length() == 0) {
               throw new IllegalArgumentException(CliStrings.CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b6de5a39/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 9bb573b..28ba856 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
@@ -394,7 +394,7 @@ public class CliStrings {
   public static final String CONNECT__MSG__CONNECTING_TO_MANAGER_AT_0 = "Connecting to Manager at {0} ..";
   public static final String CONNECT__MSG__CONNECTING_TO_MANAGER_HTTP_SERVICE_AT_0 = "Connecting to Manager's HTTP service at {0} ..";
   public static final String CONNECT__MSG__LOCATOR_COULD_NOT_FIND_MANAGER = "Locator could not find a JMX Manager";
-  public static final String CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED = "jmx password must be specified.";
+  public static final String CONNECT__MSG__JMX_PASSWORD_MUST_BE_SPECIFIED = "password must be specified.";
   public static final String CONNECT__MSG__ALREADY_CONNECTED = "Already connected to: {0}";
   public static final String CONNECT__MSG__SUCCESS = "Successfully connected to: {0}";
   public static final String CONNECT__MSG__ERROR = "Could not connect to : {0}. {1}";


[37/50] [abbrv] incubator-geode git commit: GEODE-1387 CI Failure: LocatorLauncherRemoteFileIntegrationTest.testStatusUsingWorkingDirectory

Posted by kl...@apache.org.
GEODE-1387 CI Failure: LocatorLauncherRemoteFileIntegrationTest.testStatusUsingWorkingDirectory

The locator status file writer and reader are out of sync. The reader
may read the empty file before writer writes to it. So write the status
to a temp file, then rename it. So that the reader reads the status file
that has content in it.


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

Branch: refs/heads/feature/GEODE-835
Commit: 21c0e24eed8b34088513a2ecc725e1a3dc71a2d8
Parents: aeb8c01
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Thu May 19 09:52:50 2016 -0700
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Thu May 19 09:52:50 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/process/ControllableProcess.java     | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21c0e24e/geode-core/src/main/java/com/gemstone/gemfire/internal/process/ControllableProcess.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/process/ControllableProcess.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/process/ControllableProcess.java
index f459aed..f21930d 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/process/ControllableProcess.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/process/ControllableProcess.java
@@ -64,12 +64,18 @@ public final class ControllableProcess {
         if (statusFile.exists()) {
           statusFile.delete();
         }
-        boolean created = statusFile.createNewFile();
+        final File statusFileTmp = new File(workingDir, processType.getStatusFileName() + ".tmp");
+        if (statusFileTmp.exists()) {
+          statusFileTmp.delete();
+        }
+        boolean created = statusFileTmp.createNewFile();
         assert created;
-        final FileWriter writer = new FileWriter(statusFile);
+        final FileWriter writer = new FileWriter(statusFileTmp);
         writer.write(state.toJson());
         writer.flush();
         writer.close();
+        boolean renamed = statusFileTmp.renameTo(statusFile);
+        assert renamed;
       }
     };
     


[08/50] [abbrv] incubator-geode git commit: GEODE-17: make sure commands tests are run in useHttp mode.

Posted by kl...@apache.org.
GEODE-17: make sure commands tests are run in useHttp mode.

* added the CommandOverHttpDUnitTest in geode-web
* added GfshCommandsOverHttpSecurityTest in geode-web
* move ConnectCommandWithHttpAndSSLDUnitTest to geode-web
* make sure ConnectCommandWithHttpAndSSLDUnitTest is running with the correct connection method
* updated the trusted.keystore with the supported algorithm
* create a SuiteRunner so that we can run tests in a suite yet have them report to its own xml


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

Branch: refs/heads/feature/GEODE-835
Commit: c79b64ff41f278ecffe0408cc4478bb6d19d3518
Parents: 97d6172
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue May 10 13:03:41 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri May 13 14:08:42 2016 -0700

----------------------------------------------------------------------
 .../SharedConfigurationEndToEndDUnitTest.java   |  12 +-
 .../internal/security/shiro/ShiroPrincipal.java |   2 +-
 .../cli/commands/CliCommandTestBase.java        |  47 +--
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  | 305 -------------------
 .../SharedConfigurationCommandsDUnitTest.java   |   8 +-
 .../security/GfshCommandsSecurityTest.java      |   9 +-
 .../src/test/resources/ssl/trusted.keystore     | Bin 1078 -> 2241 bytes
 .../test/junit/runner/SuiteBlockRunner.java     |  46 +++
 .../gemfire/test/junit/runner/SuiteRunner.java  |  53 ++++
 .../ClusterConfigurationDUnitTest.java          |  10 +-
 geode-web/build.gradle                          |  11 +
 .../cli/commands/CommandOverHttpDUnitTest.java  |  58 ++++
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  | 305 +++++++++++++++++++
 .../GfshCommandsOverHttpSecurityTest.java       |  29 ++
 14 files changed, 528 insertions(+), 367 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/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 caee9ea..a6221e9 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
@@ -36,10 +36,6 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.commons.io.FileUtils;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
@@ -61,6 +57,10 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
 
@@ -116,7 +116,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
   public void testStartServerAndExecuteCommands() throws Exception {
     final HeadlessGfsh gfsh = new HeadlessGfsh("gfsh2", 300, this.gfshDir);
     assertNotNull(gfsh);
-    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    connect(jmxHost, jmxPort, httpPort, gfsh);
 
     serverNames.addAll(startServers(gfsh, locatorString, 2, "Server", 1));
     doCreateCommands();
@@ -380,7 +380,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     int jmxPort = (Integer)result[2];
     int httpPort = (Integer)result[3];
 
-    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    connect(jmxHost, jmxPort, httpPort, gfsh);
 
     // Create a cache in VM 1
     VM dataMember = getHost(0).getVM(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
index 8413ebb..621eb87 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/ShiroPrincipal.java
@@ -30,7 +30,7 @@ public class ShiroPrincipal implements Principal {
 
   @Override
   public String getName() {
-    return subject.toString();
+    return subject.getPrincipal().toString();
   }
 
   public Subject getSubject(){

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 4b25165..a0966f9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -39,7 +39,6 @@ import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 
@@ -57,11 +56,8 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
   private transient HeadlessGfsh shell;
 
-  protected boolean useHttpOnConnect = false;
-  protected boolean enableAuth = false;
-  protected String jsonAuthorization = "cacheServer.json";
-  protected String username = "super-user";
-  protected String password = "1234567";
+  public static final String USE_HTTP_SYSTEM_PROPERTY = "useHTTP";
+  private boolean useHttpOnConnect = Boolean.getBoolean(USE_HTTP_SYSTEM_PROPERTY);
 
   private transient int httpPort;
   private transient int jmxPort;
@@ -107,12 +103,12 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
   @SuppressWarnings("serial")
   protected HeadlessGfsh setUpJmxManagerOnVm0ThenConnect(final Properties props) {
     setUpJMXManagerOnVM(0, props);
-    shellConnect();
+    connect(this.jmxHost, this.jmxPort, this.httpPort, getDefaultShell());
     return shell;
   }
 
   protected void setUpJMXManagerOnVM(int vm, final Properties props) {
-    Object[] result = (Object[]) Host.getHost(0).getVM(vm).invoke("setUpJmxManagerOnVm0ThenConnect", () -> {
+    Object[] result = (Object[]) Host.getHost(0).getVM(vm).invoke("setUpJmxManagerOnVm"+vm, () -> {
       final Object[] results = new Object[3];
       final Properties localProps = (props != null ? props : new Properties());
 
@@ -138,14 +134,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
       localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
       localProps.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
 
-      if (enableAuth) {
-        localProps.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
-          JSONAuthorization.class.getName() + ".create");
-        localProps.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
-
-        JSONAuthorization.setUpWithJsonFile(jsonAuthorization);
-      }
-
       getSystem(localProps);
       verifyManagementServiceStarted(getCache());
 
@@ -214,33 +202,9 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
     }
   }
 
-  /**
-   * Connect a shell to the JMX server at the given host and port
-   *
-   *
-   * @param host    Host of the JMX server
-   * @param jmxPort Port of the JMX server
-   * @param shell   Shell to connect
-   */
-  protected void shellConnect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell) {
-    assertTrue(host != null);
-    assertTrue(shell != null);
-
-    connect(host, jmxPort, httpPort, shell);
-  }
-
-  protected CommandResult shellConnect(){
-    return connect(this.jmxHost, this.jmxPort, this.httpPort, getDefaultShell());
-  }
-
-  protected CommandResult connect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell){
+  protected void connect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell){
     final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CONNECT);
 
-    if(enableAuth) {
-      command.addOption(CliStrings.CONNECT__USERNAME, username);
-      command.addOption(CliStrings.CONNECT__PASSWORD, password);
-    }
-
     String endpoint;
     if (useHttpOnConnect) {
       endpoint = "http://" + host + ":" + httpPort + "/gemfire/v1";
@@ -261,7 +225,6 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
     info("Successfully connected to managing node using " + (useHttpOnConnect ? "HTTP" : "JMX"));
     assertEquals(true, shell.isConnectedAndReady());
-    return result;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
deleted file mode 100644
index 2e0897d..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * 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.management.internal.cli.commands;
-
-import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
-import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.util.test.TestUtil.*;
-
-import java.io.File;
-import java.util.Properties;
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSession;
-
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
-import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
-import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import com.gemstone.gemfire.test.junit.categories.SecurityTest;
-
-/**
- * @since  8.1
- */
-@Category({ DistributedTest.class, SecurityTest.class })
-public class ConnectCommandWithHttpAndSSLDUnitTest extends CliCommandTestBase {
-
-  private static final ThreadLocal<Properties> sslInfoHolder = new ThreadLocal<>();
-
-  private File jks;
-
-  // TODO: should this test use @RunWith(Parameterized.class)?
-
-  @Override
-  public final void postSetUpCliCommandTestBase() throws Exception {
-    this.jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
-  }
-  
-  @Override
-  protected final void preTearDownCliCommandTestBase() throws Exception {
-    destroyDefaultSetup();
-  }
-  
-  @Override
-  public final void postTearDownCacheTestCase() throws Exception {
-    sslInfoHolder.set(null);
-  }
-
-  @Test
-  public void testMutualAuthentication() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL");
-    localProps.setProperty(HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__KEY_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__KEY_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "SSL");
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSimpleSSL() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithoutKeyStoreType() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-  
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithSSLProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLS");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSv11Protocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.1");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testSSLWithTLSv12Protocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Test
-  public void testWithMultipleProtocol() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL,TLSv1.2");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Ignore("disabled for unknown reason")
-  @Test
-  public void testSSLWithCipherSuite() throws Exception {
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
-
-    //Its bad to hard code here. But using SocketFactory.getDefaultCiphers() somehow is not working with the option 
-    //"https.cipherSuites" which is required to restrict cipher suite with HttpsURLConnection
-    //Keeping the below code for further investigation on different Java versions ( 7 & 8) @TODO
-    
-   /*SSLContext sslContext = SSLContext.getInstance("TLSv1.2");
-    
-    sslContext.init(null, null, new java.security.SecureRandom());
-    String[] cipherSuites = sslContext.getSocketFactory().getSupportedCipherSuites();*/
-    
-    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
-
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_CIPHERS, "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Ignore("disabled for unknown reason")
-  @Test
-  public void testSSLWithMultipleCipherSuite() throws Exception {
-    System.setProperty("javax.net.debug", "ssl,handshake,failure");
-    
-    Properties localProps = new Properties();
-    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
-    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
-    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256,TLS_EMPTY_RENEGOTIATION_INFO_SCSV");
-    
-    Properties clientProps = new Properties();
-    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
-    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
-    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
-    
-    sslInfoHolder.set(clientProps);
-    setUpJmxManagerOnVm0ThenConnect(localProps);
-  }
-
-  @Override
-  protected void shellConnect(final String host, final int jmxPort, final int httpPort, final HeadlessGfsh shell) {
-    assertNotNull(host);
-    assertNotNull(shell);
-
-    final CommandStringBuilder command = new CommandStringBuilder(CONNECT);
-    String endpoint;
-
-    // This is for testing purpose only. If we remove this piece of code we will
-    // get a java.security.cert.CertificateException
-    // as matching hostname can not be obtained in all test environment.
-    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
-      @Override
-      public boolean verify(String string, SSLSession ssls) {
-        return true;
-      }
-    });
-    
-    endpoint = "https://" + host + ":" + httpPort + "/gemfire/v1";
-    
-    command.addOption(CONNECT__USE_HTTP, Boolean.TRUE.toString());
-    command.addOption(CONNECT__URL, endpoint);
-    command.addOption(CONNECT__USE_SSL,Boolean.TRUE.toString());
-
-    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE) != null){
-      command.addOption(CONNECT__KEY_STORE, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD) != null){
-      command.addOption(CONNECT__KEY_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE) != null){
-      command.addOption(CONNECT__TRUST_STORE, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD) != null){
-      command.addOption(CONNECT__TRUST_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS) != null){
-      command.addOption(CONNECT__SSL_PROTOCOLS, sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS));
-    }
-    if(sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS) != null){
-      command.addOption(CONNECT__SSL_CIPHERS, sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS));
-    }
-
-    CommandResult result = executeCommand(shell, command.toString());
-
-    if (!shell.isConnectedAndReady()) {
-      fail("Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
-    }
-
-    info("Successfully connected to managing node using HTTPS");
-    assertEquals(true, shell.isConnectedAndReady());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
index 781ed98..334d2e1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
@@ -31,9 +31,6 @@ import java.io.IOException;
 import java.util.Properties;
 import java.util.Set;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -54,6 +51,9 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 /**
  * DUnit test to test export and import of shared configuration.
  */
@@ -167,7 +167,7 @@ public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
     });
 
     HeadlessGfsh gfsh = getDefaultShell();
-    shellConnect(locator1JmxHost, locator1JmxPort, locator1HttpPort, gfsh);
+    connect(locator1JmxHost, locator1JmxPort, locator1HttpPort, gfsh);
 
     // Create a cache in VM 1
     VM dataMember = getHost(0).getVM(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index d96c96c..8eaaf6a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -39,9 +39,9 @@ import org.junit.experimental.categories.Category;
 
 @Category(IntegrationTest.class)
 public class GfshCommandsSecurityTest {
-  private static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
-  private static int jmxPort = ports[0];
-  private static int httpPort = ports[1];
+  protected static int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
+  protected static int jmxPort = ports[0];
+  protected static int httpPort = ports[1];
 
   private HeadlessGfsh gfsh = null;
 
@@ -50,12 +50,13 @@ public class GfshCommandsSecurityTest {
       jmxPort, httpPort, "cacheServer.json");
 
   @Rule
-  public GfshShellConnectionRule gfshConnection = null;
+  public GfshShellConnectionRule gfshConnection;
 
   public GfshCommandsSecurityTest(){
     gfshConnection = new GfshShellConnectionRule(jmxPort, httpPort, false);
   }
 
+
   @Before
   public void before(){
     gfsh = gfshConnection.getGfsh();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-core/src/test/resources/ssl/trusted.keystore
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/ssl/trusted.keystore b/geode-core/src/test/resources/ssl/trusted.keystore
old mode 100755
new mode 100644
index 6057e3c..bd75039
Binary files a/geode-core/src/test/resources/ssl/trusted.keystore and b/geode-core/src/test/resources/ssl/trusted.keystore differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java
new file mode 100644
index 0000000..60fed84
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteBlockRunner.java
@@ -0,0 +1,46 @@
+/*
+ * 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.test.junit.runner;
+
+import org.junit.runners.BlockJUnit4ClassRunner;
+import org.junit.runners.model.FrameworkMethod;
+import org.junit.runners.model.InitializationError;
+
+/**
+ * used by SuiteRunner to override the test method name
+ */
+public class SuiteBlockRunner extends BlockJUnit4ClassRunner {
+
+  private Class<?> suiteClass;
+
+  /**
+   * Creates a BlockJUnit4ClassRunner to run {@code klass}
+   * @param klass
+   * @throws InitializationError if the test class is malformed.
+   */
+  public SuiteBlockRunner(final Class parentClass,  final Class<?> klass) throws InitializationError {
+    super(klass);
+    this.suiteClass = parentClass;
+  }
+
+  @Override
+  protected String testName(FrameworkMethod method) {
+    return method.getName()+"@"+ suiteClass.getName();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java
new file mode 100644
index 0000000..2a860c0
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/runner/SuiteRunner.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.junit.runner;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.runner.Runner;
+import org.junit.runners.Suite;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.RunnerBuilder;
+
+/**
+ * SuiteRunner is like Junit Suite, it's used in conjunction with <code>SuiteClass({xx.class, yy.class})</code>
+ * It's different from Suite in two ways:
+ * 1. it should only contain contain Junit4 test classes
+ * 2. the test method names inside each test class are appended with the suiteClass name so that the result will show up different
+ * as when you run these tests alone.
+ */
+public class SuiteRunner extends Suite {
+
+  public SuiteRunner(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
+    super(klass, getRunners(klass));
+  }
+
+  private static List<Runner> getRunners(final Class<?> klass) throws InitializationError{
+    SuiteClasses annotation = klass.getAnnotation(SuiteClasses.class);
+    if (annotation == null) {
+      throw new InitializationError(String.format("class '%s' must have a SuiteClasses annotation", klass.getName()));
+    }
+    Class<?>[] childClasses = annotation.value();
+    List<Runner> runners = new ArrayList<>();
+    for(Class childClass:childClasses){
+      runners.add(new SuiteBlockRunner(klass, childClass));
+    }
+    return runners;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
index 62bb793..d8692ce 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
@@ -40,10 +40,6 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
@@ -80,6 +76,10 @@ import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
 
@@ -663,7 +663,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
     int jmxPort = (Integer)result[2];
     int httpPort = (Integer)result[3];
     
-    shellConnect(jmxHost, jmxPort, httpPort, gfsh);
+    connect(jmxHost, jmxPort, httpPort, gfsh);
 
     final String dataMemberWorkingDir = this.temporaryFolder.getRoot().getCanonicalPath() + File.separator + dataMember;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-web/build.gradle
----------------------------------------------------------------------
diff --git a/geode-web/build.gradle b/geode-web/build.gradle
index 5bd1d89..228751b 100755
--- a/geode-web/build.gradle
+++ b/geode-web/build.gradle
@@ -52,6 +52,9 @@ dependencies {
   testCompile project(':geode-junit')
   testRuntime project(':geode-common')
   testRuntime project(':geode-core')
+  testCompile files(project(':geode-core').sourceSets.test.output)
+
+  testRuntime files(war.destinationDir)
 }
 
 //Remove the gradle output directories from the eclipse classpath. These
@@ -69,6 +72,14 @@ idea.module.iml {
   }
 }
 
+distributedTest {
+  dependsOn ':geode-core:webJar', 'war'
+}
+
+integrationTest {
+  dependsOn ':geode-core:webJar', 'war'
+}
+
 war {
   classpath configurations.runtime 
   classpath project(':geode-core').webJar.archivePath

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
new file mode 100644
index 0000000..9687eb7
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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.management.internal.cli.commands;
+
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.runner.SuiteRunner;
+
+import org.junit.ClassRule;
+import org.junit.contrib.java.lang.system.ProvideSystemProperty;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@Category(DistributedTest.class)
+@RunWith(SuiteRunner.class)
+@Suite.SuiteClasses({
+  ConfigCommandsDUnitTest.class,
+  DeployCommandsDUnitTest.class,
+  DiskStoreCommandsDUnitTest.class,
+  FunctionCommandsDUnitTest.class,
+  GemfireDataCommandsDUnitTest.class,
+  GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.class,
+  IndexCommandsDUnitTest.class,
+  ListAndDescribeDiskStoreCommandsDUnitTest.class,
+  ListIndexCommandDUnitTest.class,
+  MemberCommandsDUnitTest.class,
+  MiscellaneousCommandsDUnitTest.class,
+  MiscellaneousCommandsExportLogsPart1DUnitTest.class,
+  MiscellaneousCommandsExportLogsPart2DUnitTest.class,
+  MiscellaneousCommandsExportLogsPart3DUnitTest.class,
+  MiscellaneousCommandsExportLogsPart4DUnitTest.class,
+  QueueCommandsDUnitTest.class,
+  SharedConfigurationCommandsDUnitTest.class,
+  ShellCommandsDUnitTest.class,
+  ShowDeadlockDUnitTest.class,
+  ShowMetricsDUnitTest.class,
+  ShowStackTraceDUnitTest.class,
+  UserCommandsDUnitTest.class
+})
+public class CommandOverHttpDUnitTest {
+  @ClassRule
+  public static ProvideSystemProperty provideSystemProperty = new ProvideSystemProperty(CliCommandTestBase.USE_HTTP_SYSTEM_PROPERTY, "true");
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
new file mode 100644
index 0000000..a8f209b
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConnectCommandWithHttpAndSSLDUnitTest.java
@@ -0,0 +1,305 @@
+/*
+ * 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.management.internal.cli.commands;
+
+import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
+import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.util.test.TestUtil.*;
+
+import java.io.File;
+import java.util.Properties;
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSession;
+
+import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * @since  8.1
+ */
+@Category({ DistributedTest.class, SecurityTest.class })
+public class ConnectCommandWithHttpAndSSLDUnitTest extends CliCommandTestBase {
+
+  private static final ThreadLocal<Properties> sslInfoHolder = new ThreadLocal<>();
+
+  private File jks;
+
+  // TODO: should this test use @RunWith(Parameterized.class)?
+
+  @Override
+  public final void postSetUpCliCommandTestBase() throws Exception {
+    this.jks = new File(getResourcePath(getClass(), "/ssl/trusted.keystore"));
+  }
+  
+  @Override
+  protected final void preTearDownCliCommandTestBase() throws Exception {
+    destroyDefaultSetup();
+  }
+  
+  @Override
+  public final void postTearDownCacheTestCase() throws Exception {
+    sslInfoHolder.set(null);
+  }
+
+  @Test
+  public void testMutualAuthentication() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
+    serverProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL");
+    serverProps.setProperty(HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME, "true");
+    serverProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_NAME, jks.getCanonicalPath());
+    serverProps.setProperty(HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
+
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__KEY_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__KEY_STORE_PASSWORD, "password");
+    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "SSL");
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(serverProps);
+  }
+
+  @Test
+  public void testSimpleSSL() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    serverProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
+
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(serverProps);
+  }
+
+  @Test
+  public void testSSLWithoutKeyStoreType() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+  
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithSSLProtocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithTLSProtocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLS");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithTLSv11Protocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.1");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testSSLWithTLSv12Protocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Test
+  public void testWithMultipleProtocol() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL,TLSv1.2");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testSSLWithCipherSuite() throws Exception {
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
+
+    //Its bad to hard code here. But using SocketFactory.getDefaultCiphers() somehow is not working with the option 
+    //"https.cipherSuites" which is required to restrict cipher suite with HttpsURLConnection
+    //Keeping the below code for further investigation on different Java versions ( 7 & 8) @TODO
+    
+   /*SSLContext sslContext = SSLContext.getInstance("TLSv1.2");
+    
+    sslContext.init(null, null, new java.security.SecureRandom());
+    String[] cipherSuites = sslContext.getSocketFactory().getSupportedCipherSuites();*/
+    
+    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
+
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    clientProps.setProperty(CONNECT__SSL_CIPHERS, "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256");
+    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testSSLWithMultipleCipherSuite() throws Exception {
+    System.setProperty("javax.net.debug", "ssl,handshake,failure");
+    
+    Properties localProps = new Properties();
+    localProps.setProperty(HTTP_SERVICE_SSL_ENABLED_NAME, "true");
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
+    localProps.setProperty(HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+    localProps.setProperty(HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    localProps.setProperty(HTTP_SERVICE_SSL_CIPHERS_NAME,"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256,TLS_EMPTY_RENEGOTIATION_INFO_SCSV");
+    
+    Properties clientProps = new Properties();
+    clientProps.setProperty(CONNECT__TRUST_STORE, jks.getCanonicalPath());
+    clientProps.setProperty(CONNECT__TRUST_STORE_PASSWORD, "password");
+    clientProps.setProperty(CONNECT__SSL_PROTOCOLS, "TLSv1.2");
+    
+    sslInfoHolder.set(clientProps);
+    setUpJmxManagerOnVm0ThenConnect(localProps);
+  }
+
+  @Override
+  protected void connect(final String host, final int jmxPort, final int httpPort, final HeadlessGfsh shell) {
+    assertNotNull(host);
+    assertNotNull(shell);
+
+    final CommandStringBuilder command = new CommandStringBuilder(CONNECT);
+    String endpoint;
+
+    // This is for testing purpose only. If we remove this piece of code we will
+    // get a java.security.cert.CertificateException
+    // as matching hostname can not be obtained in all test environment.
+    HttpsURLConnection.setDefaultHostnameVerifier(new HostnameVerifier() {
+      @Override
+      public boolean verify(String string, SSLSession ssls) {
+        return true;
+      }
+    });
+    
+    endpoint = "https://" + host + ":" + httpPort + "/gemfire/v1";
+    
+    command.addOption(CONNECT__USE_HTTP, Boolean.TRUE.toString());
+    command.addOption(CONNECT__URL, endpoint);
+    command.addOption(CONNECT__USE_SSL,Boolean.TRUE.toString());
+
+    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE) != null){
+      command.addOption(CONNECT__KEY_STORE, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD) != null){
+      command.addOption(CONNECT__KEY_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__KEY_STORE_PASSWORD));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE) != null){
+      command.addOption(CONNECT__TRUST_STORE, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD) != null){
+      command.addOption(CONNECT__TRUST_STORE_PASSWORD, sslInfoHolder.get().getProperty(CONNECT__TRUST_STORE_PASSWORD));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS) != null){
+      command.addOption(CONNECT__SSL_PROTOCOLS, sslInfoHolder.get().getProperty(CONNECT__SSL_PROTOCOLS));
+    }
+    if(sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS) != null){
+      command.addOption(CONNECT__SSL_CIPHERS, sslInfoHolder.get().getProperty(CONNECT__SSL_CIPHERS));
+    }
+
+    CommandResult result = executeCommand(shell, command.toString());
+
+    if (!shell.isConnectedAndReady()) {
+      fail("Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
+    }
+
+    info("Successfully connected to managing node using HTTPS");
+    assertEquals(true, shell.isConnectedAndReady());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c79b64ff/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java
new file mode 100644
index 0000000..af0cfad
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsOverHttpSecurityTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.management.internal.security;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class GfshCommandsOverHttpSecurityTest extends GfshCommandsSecurityTest {
+  public GfshCommandsOverHttpSecurityTest(){
+    gfshConnection = new GfshShellConnectionRule(jmxPort, httpPort, true);
+  }
+}


[34/50] [abbrv] incubator-geode git commit: GEODE-1292: change defragment to not create fragments > 2G

Posted by kl...@apache.org.
GEODE-1292: change defragment to not create fragments > 2G


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

Branch: refs/heads/feature/GEODE-835
Commit: d14af16ae7ed4d52379877e508683b6df05ff5d1
Parents: 3a37159
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon May 16 16:12:46 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed May 18 13:49:44 2016 -0700

----------------------------------------------------------------------
 .../internal/offheap/FreeListManager.java       | 292 +++++++++++--------
 .../OffHeapStoredObjectAddressStack.java        |   5 +-
 .../gemfire/internal/offheap/SlabImpl.java      |  11 +-
 .../internal/offheap/FreeListManagerTest.java   |  96 +++++-
 4 files changed, 280 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d14af16a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
index c5e17e8..9940b7d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/FreeListManager.java
@@ -301,6 +301,44 @@ public class FreeListManager {
     verifyHugeMultiple(HUGE_MULTIPLE);
   }
   public final static int MAX_TINY = TINY_MULTIPLE*TINY_FREE_LIST_COUNT;
+  
+  /**
+   * Return true if the two chunks have been combined into one.
+   * If low and high are adjacent to each other
+   * and the combined size is small enough (see isSmallEnough)
+   * then low's size will be increased by the size of high
+   * and true will be returned.
+   */
+  boolean combineIfAdjacentAndSmallEnough(long lowAddr, long highAddr) {
+    assert lowAddr <= highAddr;
+    int lowSize = OffHeapStoredObject.getSize(lowAddr);
+    if (isAdjacent(lowAddr, lowSize, highAddr)) {
+      int highSize = OffHeapStoredObject.getSize(highAddr);
+      int combinedSize = lowSize + highSize;
+      if (isSmallEnough(combinedSize)) {
+        // append the highAddr chunk to lowAddr
+        OffHeapStoredObject.setSize(lowAddr, (int)combinedSize);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns true if the area if memory (starting at lowAddr and extending to
+   * lowAddr+lowSize) is right before (i.e. adjacent) to highAddr.
+   */
+  boolean isAdjacent(long lowAddr, int lowSize, long highAddr) {
+    return (lowAddr + lowSize) == highAddr;
+  }
+  /**
+   * Return true if size is small enough to be set as the size
+   * of a OffHeapStoredObject.
+   */
+  boolean isSmallEnough(long size) {
+    return size <= Integer.MAX_VALUE;
+  }
+  
   /**
    * Defragments memory and returns true if enough memory to allocate chunkSize
    * is freed. Otherwise returns false;
@@ -316,128 +354,150 @@ public class FreeListManager {
           // So just return true causing the caller to retry the allocation.
           return true;
         }
-        ArrayList<OffHeapStoredObjectAddressStack> freeChunks = new ArrayList<OffHeapStoredObjectAddressStack>();
-        collectFreeChunks(freeChunks);
-        final int SORT_ARRAY_BLOCK_SIZE = 128;
-        long[] sorted = new long[SORT_ARRAY_BLOCK_SIZE];
-        int sortedSize = 0;
-        boolean result = false;
-        int largestFragment = 0;
-        for (OffHeapStoredObjectAddressStack l: freeChunks) {
-          long addr = l.poll();
-          while (addr != 0) {
-            int idx = Arrays.binarySearch(sorted, 0, sortedSize, addr);
-            idx = -idx;
-            idx--;
-            if (idx == sortedSize) {
-              // addr is > everything in the array
-              if (sortedSize == 0) {
-                // nothing was in the array
-                sorted[0] = addr;
-                sortedSize++;
-              } else {
-                // see if we can conflate into sorted[idx]
-                long lowAddr = sorted[idx-1];
-                int lowSize = OffHeapStoredObject.getSize(lowAddr);
-                if (lowAddr + lowSize == addr) {
-                  // append the addr chunk to lowAddr
-                  OffHeapStoredObject.setSize(lowAddr, lowSize + OffHeapStoredObject.getSize(addr));
-                } else {
-                  if (sortedSize >= sorted.length) {
-                    long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
-                    System.arraycopy(sorted, 0, newSorted, 0, sorted.length);
-                    sorted = newSorted;
-                  }
-                  sortedSize++;
-                  sorted[idx] = addr;
-                }
-              }
-            } else {
-              int addrSize = OffHeapStoredObject.getSize(addr);
-              long highAddr = sorted[idx];
-              if (addr + addrSize == highAddr) {
-                // append highAddr chunk to addr
-                OffHeapStoredObject.setSize(addr, addrSize + OffHeapStoredObject.getSize(highAddr));
-                sorted[idx] = addr;
-              } else {
-                boolean insert = idx==0;
-                if (!insert) {
-                  long lowAddr = sorted[idx-1];
-                  //                  if (lowAddr == 0L) {
-                  //                    long[] tmp = Arrays.copyOf(sorted, sortedSize);
-                  //                    throw new IllegalStateException("addr was zero at idx=" + (idx-1) + " sorted="+ Arrays.toString(tmp));
-                  //                  }
-                  int lowSize = OffHeapStoredObject.getSize(lowAddr);
-                  if (lowAddr + lowSize == addr) {
-                    // append the addr chunk to lowAddr
-                    OffHeapStoredObject.setSize(lowAddr, lowSize + addrSize);
-                  } else {
-                    insert = true;
-                  }
-                }
-                if (insert) {
-                  if (sortedSize >= sorted.length) {
-                    long[] newSorted = new long[sorted.length+SORT_ARRAY_BLOCK_SIZE];
-                    System.arraycopy(sorted, 0, newSorted, 0, idx);
-                    newSorted[idx] = addr;
-                    System.arraycopy(sorted, idx, newSorted, idx+1, sortedSize-idx);
-                    sorted = newSorted;
-                  } else {
-                    System.arraycopy(sorted, idx, sorted, idx+1, sortedSize-idx);
-                    sorted[idx] = addr;
-                  }
-                  sortedSize++;
-                }
-              }
-            }
-            addr = l.poll();
-          }
-        }
-        for (int i=sortedSize-1; i > 0; i--) {
-          long addr = sorted[i];
-          long lowAddr = sorted[i-1];
-          int lowSize = OffHeapStoredObject.getSize(lowAddr);
-          if (lowAddr + lowSize == addr) {
-            // append addr chunk to lowAddr
-            OffHeapStoredObject.setSize(lowAddr, lowSize + OffHeapStoredObject.getSize(addr));
-            sorted[i] = 0L;
-          }
-        }
-        this.lastFragmentAllocation.set(0);
-        ArrayList<Fragment> tmp = new ArrayList<Fragment>();
-        for (int i=sortedSize-1; i >= 0; i--) {
-          long addr = sorted[i];
-          if (addr == 0L) continue;
-          int addrSize = OffHeapStoredObject.getSize(addr);
-          Fragment f = createFragment(addr, addrSize);
-          if (addrSize >= chunkSize) {
-            result = true;
-          }
-          if (addrSize > largestFragment) {
-            largestFragment = addrSize;
-            // TODO it might be better to sort them biggest first
-            tmp.add(0, f);
-          } else {
-            tmp.add(f);
-          }
-        }
-        this.fragmentList.addAll(tmp);
-
-        fillFragments();
+        boolean result = doDefragment(chunkSize);
 
         // Signal any waiters that a defragmentation happened.
         this.defragmentationCount.incrementAndGet();
 
-        this.ma.getStats().setLargestFragment(largestFragment);
-        this.ma.getStats().setFragments(tmp.size());        
-        this.ma.getStats().setFragmentation(getFragmentation());
-
         return result;
       } // sync
     } finally {
       this.ma.getStats().endDefragmentation(startDefragmentationTime);
     }
   }
+  
+  /**
+   * Simple interface the represents a "stack" of primitive longs.
+   * Currently this interface only allows supports poll but more
+   * could be added if needed in the future.
+   * This interface was introduced to aid unit testing.
+   * The only implementation of it is OffHeapStoredObjectAddressStack.
+   */
+  public interface LongStack {
+    /**
+     * Retrieves and removes the top of this stack,
+     * or returns {@code 0L} if this stack is empty.
+     */
+    public long poll();
+  }
+  /**
+   * Manages an array of primitive longs. The array can grow.
+   */
+  public static class ResizableLongArray {
+    private static final int SORT_ARRAY_BLOCK_SIZE = 128;
+    long[] data = new long[SORT_ARRAY_BLOCK_SIZE];
+    int size = 0;
+    
+    public int binarySearch(long l) {
+      return Arrays.binarySearch(data, 0, size, l);
+    }
+    public int size() {
+      return size;
+    }
+    public long get(int idx) {
+      return data[idx];
+    }
+    public void set(int idx, long l) {
+      data[idx] = l;
+    }
+    public void add(long l) {
+      if (size >= data.length) {
+        long[] newData = new long[data.length+SORT_ARRAY_BLOCK_SIZE];
+        System.arraycopy(data, 0, newData, 0, data.length);
+        data = newData;
+      }
+      data[size] = l;
+      size++;
+    }
+    public void insert(int idx, long l) {
+      if (size >= data.length) {
+        long[] newData = new long[data.length+SORT_ARRAY_BLOCK_SIZE];
+        System.arraycopy(data, 0, newData, 0, idx);
+        newData[idx] = l;
+        System.arraycopy(data, idx, newData, idx+1, size-idx);
+        data = newData;
+      } else {
+        System.arraycopy(data, idx, data, idx+1, size-idx);
+        data[idx] = l;
+      }
+      size++;
+    }
+  }
+  /**
+   * Defragments memory and returns true if enough memory to allocate chunkSize
+   * is freed. Otherwise returns false;
+   * Unlike the defragment method this method is not thread safe and does not check
+   * for a concurrent defragment. It should only be called by defragment and unit tests.
+   */
+  boolean doDefragment(int chunkSize) {
+    boolean result = false;
+    ArrayList<LongStack> freeChunks = new ArrayList<LongStack>();
+    collectFreeChunks(freeChunks);
+    ResizableLongArray sorted = new ResizableLongArray();
+    for (LongStack l: freeChunks) {
+      long addr = l.poll();
+      while (addr != 0) {
+        int idx = sorted.binarySearch(addr);
+        idx = -idx;
+        idx--;
+        int sortedSize = sorted.size();
+        if (idx == sortedSize) {
+          // addr is > everything in the array
+          if (sortedSize == 0) {
+            // nothing was in the array
+            sorted.add(addr);
+          } else {
+            if (!combineIfAdjacentAndSmallEnough(sorted.get(idx-1), addr)) {
+              sorted.add(addr);
+            }
+          }
+        } else {
+          if (combineIfAdjacentAndSmallEnough(addr, sorted.get(idx))) {
+            sorted.set(idx, addr);
+          } else {
+            if (idx == 0 || !combineIfAdjacentAndSmallEnough(sorted.get(idx-1), addr)) {
+              sorted.insert(idx, addr);
+            }
+          }
+        }
+        addr = l.poll();
+      }
+    }
+    for (int i=sorted.size()-1; i > 0; i--) {
+      if (combineIfAdjacentAndSmallEnough(sorted.get(i-1), sorted.get(i))) {
+        sorted.set(i, 0L);
+      }
+    }
+    
+    int largestFragment = 0;
+    this.lastFragmentAllocation.set(0);
+    ArrayList<Fragment> tmp = new ArrayList<Fragment>();
+    for (int i=sorted.size()-1; i >= 0; i--) {
+      long addr = sorted.get(i);
+      if (addr == 0L) continue;
+      int addrSize = OffHeapStoredObject.getSize(addr);
+      Fragment f = createFragment(addr, addrSize);
+      if (addrSize >= chunkSize) {
+        result = true;
+      }
+      if (addrSize > largestFragment) {
+        largestFragment = addrSize;
+        // TODO it might be better to sort them biggest first
+        tmp.add(0, f);
+      } else {
+        tmp.add(f);
+      }
+    }
+    this.fragmentList.addAll(tmp);
+
+    fillFragments();
+
+    this.ma.getStats().setLargestFragment(largestFragment);
+    this.ma.getStats().setFragments(tmp.size());        
+    this.ma.getStats().setFragmentation(getFragmentation());
+
+    return result;
+  }
 
   /**
    * Unit tests override this method to get better test coverage
@@ -493,7 +553,7 @@ public class FreeListManager {
     }
   }
 
-  private void collectFreeChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeChunks(List<LongStack> l) {
     collectFreeFragmentChunks(l);
     collectFreeHugeChunks(l);
     collectFreeTinyChunks(l);
@@ -501,7 +561,7 @@ public class FreeListManager {
   List<Fragment> getFragmentList() {
     return this.fragmentList;
   }
-  private void collectFreeFragmentChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeFragmentChunks(List<LongStack> l) {
     if (this.fragmentList.size() == 0) return;
     OffHeapStoredObjectAddressStack result = new OffHeapStoredObjectAddressStack();
     for (Fragment f: this.fragmentList) {
@@ -530,7 +590,7 @@ public class FreeListManager {
       l.add(result);
     }
   }
-  private void collectFreeTinyChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeTinyChunks(List<LongStack> l) {
     for (int i=0; i < this.tinyFreeLists.length(); i++) {
       OffHeapStoredObjectAddressStack cl = this.tinyFreeLists.get(i);
       if (cl != null) {
@@ -541,7 +601,7 @@ public class FreeListManager {
       }
     }
   }
-  private void collectFreeHugeChunks(List<OffHeapStoredObjectAddressStack> l) {
+  private void collectFreeHugeChunks(List<LongStack> l) {
     OffHeapStoredObject c = this.hugeChunkSet.pollFirst();
     OffHeapStoredObjectAddressStack result = null;
     while (c != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d14af16a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
index b69d3a6..1719b1f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapStoredObjectAddressStack.java
@@ -18,13 +18,15 @@ package com.gemstone.gemfire.internal.offheap;
 
 import org.apache.logging.log4j.Logger;
 
+import com.gemstone.gemfire.internal.offheap.FreeListManager.LongStack;
+
 /**
  * A "stack" of addresses of OffHeapStoredObject instances. The stored objects are not kept
  * in java object form but instead each one is just an off-heap address.
  * This class is used for each "tiny" free-list of the FreeListManager.
  * This class is thread safe.
  */
-public class OffHeapStoredObjectAddressStack {
+public class OffHeapStoredObjectAddressStack implements LongStack {
   // Ok to read without sync but must be synced on write
   private volatile long topAddr;
   
@@ -46,6 +48,7 @@ public class OffHeapStoredObjectAddressStack {
       this.topAddr = e;
     }
   }
+  @Override
   public long poll() {
     long result;
     synchronized (this) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d14af16a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
index 1c88bde..3169352 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/SlabImpl.java
@@ -26,7 +26,16 @@ public class SlabImpl implements Slab {
   private final int size;
   
   public SlabImpl(int size) {
-    this(AddressableMemoryManager.allocate(size), size);
+    this(size, false);
+  }
+
+  /**
+   * This constructor is used by unit test to add padding
+   * to prevent intermittent combining of fragments during
+   * defragmentation.
+   */
+  public SlabImpl(int size, boolean withPadding) {
+    this(AddressableMemoryManager.allocate(size + (withPadding ? 4 : 0)), size);
   }
 
   public SlabImpl(long addr, int size) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d14af16a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
index 4cf1df0..9c6a3db 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/offheap/FreeListManagerTest.java
@@ -57,6 +57,10 @@ public class FreeListManagerTest {
     return new TestableFreeListManager(ma, slabs);
   }
   
+  private static TestableFreeListManager createFreeListManager(MemoryAllocatorImpl ma, Slab[] slabs, int maxCombine) {
+    return new TestableFreeListManager(ma, slabs, maxCombine);
+  }
+  
   private void setUpSingleSlabManager() {
     setUpSingleSlabManager(DEFAULT_SLAB_SIZE);
   }
@@ -285,14 +289,22 @@ public class FreeListManagerTest {
   }
   
   @Test
+  public void testSlabImplToString() {
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    String slabAsString = slab.toString();
+    assertThat(slabAsString.contains("MemoryAddress="+slab.getMemoryAddress()));
+    assertThat(slabAsString.contains("Size="+DEFAULT_SLAB_SIZE));
+  }
+  
+  @Test
   public void defragmentWithChunkSizeOfMaxSlabReturnsTrue() {
     int SMALL_SLAB = 16;
     int MEDIUM_SLAB = 128;
-    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE, true);
     this.freeListManager = createFreeListManager(ma, new Slab[] {
-        new SlabImpl(SMALL_SLAB), 
-        new SlabImpl(SMALL_SLAB), 
-        new SlabImpl(MEDIUM_SLAB), 
+        new SlabImpl(SMALL_SLAB, true), 
+        new SlabImpl(SMALL_SLAB, true), 
+        new SlabImpl(MEDIUM_SLAB, true), 
         slab});
     ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
     chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
@@ -303,8 +315,9 @@ public class FreeListManagerTest {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
+    this.freeListManager.firstDefragmentation = false;
     assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE)).isTrue();
-    //assertThat(this.freeListManager.getFragmentList()).hasSize(4); // TODO intermittently fails because Fragments may be merged
+    assertThat(this.freeListManager.getFragmentList()).hasSize(4);
   }
   
   @Test
@@ -326,6 +339,31 @@ public class FreeListManagerTest {
       OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
     }
     
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE/2)).isTrue();
+  }
+  
+  @Test
+  public void defragmentWhenDisallowingCombine() {
+    int SMALL_SLAB = 16;
+    int MEDIUM_SLAB = 128;
+    Slab slab = new SlabImpl(DEFAULT_SLAB_SIZE);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(SMALL_SLAB), 
+        new SlabImpl(MEDIUM_SLAB), 
+        slab}, DEFAULT_SLAB_SIZE/2);
+    ArrayList<OffHeapStoredObject> chunks = new ArrayList<>();
+    chunks.add(this.freeListManager.allocate(SMALL_SLAB-8+1));
+    chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
+    chunks.add(this.freeListManager.allocate(DEFAULT_SLAB_SIZE/2-8));
+    this.freeListManager.allocate(SMALL_SLAB-8+1);
+    for (OffHeapStoredObject c: chunks) {
+      OffHeapStoredObject.release(c.getAddress(), this.freeListManager);
+    }
+    
+    this.freeListManager.firstDefragmentation = false;
+    assertThat(this.freeListManager.defragment((DEFAULT_SLAB_SIZE/2)+1)).isFalse();
     assertThat(this.freeListManager.defragment(DEFAULT_SLAB_SIZE/2)).isTrue();
   }
   
@@ -335,7 +373,7 @@ public class FreeListManagerTest {
     OffHeapStoredObject c = freeListManager.allocate(DEFAULT_SLAB_SIZE-8);
     this.freeListManager.firstDefragmentation = false;
     assertThat(this.freeListManager.defragment(1)).isFalse();
-    // call defragmen twice for extra code coverage
+    // call defragment twice for extra code coverage
     assertThat(this.freeListManager.defragment(1)).isFalse();
     assertThat(this.freeListManager.getFragmentList()).isEmpty();
   }
@@ -430,6 +468,7 @@ public class FreeListManagerTest {
     this.freeListManager.allocate(DEFAULT_SLAB_SIZE-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     this.freeListManager.allocate(MEDIUM_SLAB-8-(OffHeapStoredObject.MIN_CHUNK_SIZE-1));
     
+    this.freeListManager.firstDefragmentation = false;
     assertThat(this.freeListManager.defragment(SMALL_SLAB)).isTrue();
   }
  @Test
@@ -806,6 +845,34 @@ public class FreeListManagerTest {
     assertThat(spy.getFragmentation()).isEqualTo(67); //Math.rint(66.66)
   }
   
+  @Test
+  public void isAdjacentBoundaryConditions() {
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    
+    assertThat(!this.freeListManager.isAdjacent(Long.MAX_VALUE-4, 4, Long.MAX_VALUE+1));
+    assertThat(this.freeListManager.isAdjacent(Long.MAX_VALUE-4, 4, Long.MAX_VALUE));
+    assertThat(this.freeListManager.isAdjacent(-8L, 4, -4L));
+    long lowAddr = Long.MAX_VALUE;
+    long highAddr = lowAddr + 4;
+    assertThat(this.freeListManager.isAdjacent(lowAddr, 4, highAddr));
+    assertThat(!this.freeListManager.isAdjacent(lowAddr, 4, highAddr-1));
+    assertThat(!this.freeListManager.isAdjacent(lowAddr, 4, highAddr+1));
+    lowAddr = highAddr;
+    highAddr = lowAddr + 4;
+    assertThat(this.freeListManager.isAdjacent(lowAddr, 4, highAddr));
+    assertThat(!this.freeListManager.isAdjacent(highAddr, 4, lowAddr));
+  }
+  @Test
+  public void isSmallEnoughBoundaryConditions() {
+    SlabImpl chunk = new SlabImpl(10);
+    this.freeListManager = createFreeListManager(ma, new Slab[] {chunk});
+    
+    assertThat(this.freeListManager.isSmallEnough(Integer.MAX_VALUE));
+    assertThat(this.freeListManager.isSmallEnough(Integer.MAX_VALUE-1));
+    assertThat(!this.freeListManager.isSmallEnough(Integer.MAX_VALUE+1L));
+    assertThat(!this.freeListManager.isSmallEnough(Long.MAX_VALUE));
+  }
   /**
    * Just like Fragment except that the first time allocate is called
    * it returns false indicating that the allocate failed.
@@ -832,9 +899,15 @@ public class FreeListManagerTest {
   private static class TestableFreeListManager extends FreeListManager {
     private boolean firstTime = true;
     private boolean firstDefragmentation = true;
+    private final int maxCombine;
 
     public TestableFreeListManager(MemoryAllocatorImpl ma, Slab[] slabs) {
+      this(ma, slabs, 0);
+    }
+
+    public TestableFreeListManager(MemoryAllocatorImpl ma, Slab[] slabs, int maxCombine) {
       super(ma, slabs);
+      this.maxCombine = maxCombine;
     }
 
     @Override
@@ -860,8 +933,19 @@ public class FreeListManagerTest {
         this.firstDefragmentation = false;
         // Force defragmentation into thinking a concurrent defragmentation happened.
         this.defragmentationCount.incrementAndGet();
+      } else {
+        super.afterDefragmentationCountFetched();
+      }
+    }
+    @Override
+    boolean isSmallEnough(long size) {
+      if (this.maxCombine != 0) {
+        return size <= this.maxCombine;
+      } else {
+        return super.isSmallEnough(size);
       }
     }
+
     
   }
 }


[16/50] [abbrv] incubator-geode git commit: GEODE-1351: add waitUntilFlush() into luceneIndexImpl

Posted by kl...@apache.org.
GEODE-1351: add waitUntilFlush() into luceneIndexImpl

This function is very useful in test and demo.
It will wait for the AEQ size to be 0

Also added junit test and integration tests.


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

Branch: refs/heads/feature/GEODE-835
Commit: 34d379196c3bf2129a391ce63a9890daa3d75b7b
Parents: e1cbc9b
Author: zhouxh <gz...@pivotal.io>
Authored: Sun May 15 21:36:34 2016 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Mon May 16 11:42:05 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndex.java       |  5 ++
 .../lucene/internal/LuceneEventListener.java    | 11 +++
 .../cache/lucene/internal/LuceneIndexImpl.java  | 43 +++++++++--
 .../internal/xml/LuceneIndexCreation.java       |  4 +
 .../gemfire/cache/lucene/LuceneQueriesBase.java | 69 ++++++++++++++++-
 .../internal/LuceneIndexImplJUnitTest.java      | 78 ++++++++++++++++++++
 6 files changed, 202 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/34d37919/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
index 743045b..be329f7 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
@@ -56,4 +56,9 @@ public interface LuceneIndex {
    */
   public Map<String, Analyzer> getFieldAnalyzers();
   
+  /* 
+   * wait until the current entries in cache are indexed
+   */
+  public void waitUntilFlushed(int maxWait);
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/34d37919/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
index 9fdfd43..2dae4ee 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
@@ -35,6 +35,8 @@ import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+import com.gemstone.gemfire.internal.cache.CacheObserverHolder;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy.TestHook;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
@@ -70,6 +72,10 @@ public class LuceneEventListener implements AsyncEventListener {
         IndexRepository repository = repositoryManager.getRepository(region, key, callbackArgument);
 
         Operation op = event.getOperation();
+        
+        if (testHook != null) {
+          testHook.doTestHook("FOUND_AND_BEFORE_PROCESSING_A_EVENT");
+        }
 
         if (op.isCreate()) {
           repository.create(key, event.getDeserializedValue());
@@ -96,4 +102,9 @@ public class LuceneEventListener implements AsyncEventListener {
       DefaultQuery.setPdxReadSerialized(false);
     }
   }
+  
+  public interface TestHook {
+    public void doTestHook(String spot);
+  }
+  public static TestHook testHook;
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/34d37919/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index 0b5f8fa..981d9e4 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -21,23 +21,26 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import java.util.Collections;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
 import com.gemstone.gemfire.cache.lucene.internal.xml.LuceneIndexCreation;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 public abstract class LuceneIndexImpl implements InternalLuceneIndex {
@@ -77,7 +80,33 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
   protected void setSearchableFields(String[] fields) {
     searchableFieldNames = fields;
   }
-  
+
+  /*
+   *  For test and demo purpose. To use it, the data region should stop feeding
+   *  A more advanced version is under-development
+   */
+  @Override
+  public void waitUntilFlushed(int maxWait) {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
+    AsyncEventQueue queue = (AsyncEventQueue)cache.getAsyncEventQueue(aeqId);
+    if (queue != null) {
+      long start = System.nanoTime();
+      while (System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(maxWait)) {
+        if (0 == queue.size()) {
+          logger.debug("waitUntilFlushed: Queue size is 0");
+          break;
+        } else {
+          try {
+            Thread.sleep(200);
+          } catch (InterruptedException e) {
+          }
+        }
+      }
+    } else { 
+      throw new IllegalArgumentException("The AEQ does not exist for the index "+indexName+" region "+regionPath);
+    }
+  }
+
   @Override
   public String[] getFieldNames() {
     return searchableFieldNames;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/34d37919/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
index 86a10e4..b54f51b 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -114,4 +114,8 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
   public void addFieldNames(String[] fieldNames) {
     this.fieldNames.addAll(Arrays.asList(fieldNames));
   }
+
+  @Override
+  public void waitUntilFlushed(int maxWait) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/34d37919/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index c467a18..c7567f3 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -18,7 +18,7 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.io.Serializable;
 import java.util.HashMap;
@@ -27,6 +27,12 @@ import java.util.Map;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneEventListener;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexImpl;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
+import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.VM;
@@ -75,6 +81,67 @@ public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
     executeTextSearch(accessor);
   }
 
+  @Test
+  public void entriesFlushedToIndexAfterWaitForFlushCalled() {
+    SerializableRunnableIF createIndex = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    };
+    dataStore1.invoke(() -> initDataStore(createIndex));
+    dataStore2.invoke(() -> initDataStore(createIndex));
+    accessor.invoke(() -> initAccessor(createIndex));
+
+    try {
+      dataStore1.invoke(() -> setTestHook());
+      putDataInRegion(accessor);
+      waitForFlushBeforeExecuteTextSearch(accessor, 10);
+      executeTextSearch(accessor);
+    } finally {
+      dataStore1.invoke(() -> checkResultAndresetTestHook());
+    }
+  }
+
+  protected void waitForFlushBeforeExecuteTextSearch(VM vm, final int expectKeyNum) {
+    vm.invoke(() -> {
+      Cache cache = getCache();
+      Region<Object, Object> region = cache.getRegion(REGION_NAME);
+
+      LuceneService service = LuceneServiceProvider.get(cache);
+      LuceneIndexImpl index = (LuceneIndexImpl)service.getIndex(INDEX_NAME, REGION_NAME);
+      assertNotNull(index);
+      LuceneQuery<Integer, TestObject> query;
+
+      String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+      AsyncEventQueue queue = cache.getAsyncEventQueue(aeqId);
+      assertNotNull(queue);
+      assertTrue(queue.size()>0);
+      index.waitUntilFlushed(30000);
+      return null;
+    });
+  }
+
+  public static void setTestHook() {
+    LuceneEventListener.testHook = new LuceneEventListener.TestHook() {
+
+      @Override
+      public void doTestHook(String spot) {
+        if (spot.equals("FOUND_AND_BEFORE_PROCESSING_A_EVENT")) {
+          try {
+            Thread.sleep(1000);
+            LogService.getLogger().debug("Waited in test hook");
+          }
+          catch (InterruptedException e) {
+          }
+        }
+      }
+    };
+  }
+  
+  public static void checkResultAndresetTestHook()
+  {
+    LuceneEventListener.testHook = null;
+  }
+
   protected void executeTextSearch(VM vm) {
     vm.invoke(() -> {
       Cache cache = getCache();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/34d37919/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
new file mode 100755
index 0000000..edecc66
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImplJUnitTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cache.lucene.internal;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import static org.mockito.Mockito.*;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.test.fake.Fakes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LuceneIndexImplJUnitTest {
+  public static final String REGION = "region";
+  public static final String INDEX = "index";
+  public static final int MAX_WAIT = 30000;
+  private Cache cache;
+  LuceneIndex index;
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+  
+  @Before
+  public void createLuceneIndex() {
+    cache = Fakes.cache();
+    index = new LuceneIndexForPartitionedRegion(INDEX, REGION, cache);
+  }
+  
+  @Test
+  public void waitUnitFlushedWithMissingAEQThrowsIllegalArgument() throws Exception {
+    thrown.expect(IllegalArgumentException.class);
+    index.waitUntilFlushed(MAX_WAIT);
+  }
+  
+  @Test
+  public void waitUnitFlushedWaitsForFlush() throws Exception {
+    final String expectedIndexName = LuceneServiceImpl.getUniqueIndexName(INDEX, REGION);
+    final AsyncEventQueue queue = mock(AsyncEventQueue.class);
+    when(cache.getAsyncEventQueue(eq(expectedIndexName))).thenReturn(queue);
+    
+    AtomicInteger callCount = new AtomicInteger();
+    when(queue.size()).thenAnswer(invocation -> {
+      if (callCount.get() == 0) {
+        // when the waitUnitFlushed() called the 2nd time, queue.size() will return 0
+        callCount.incrementAndGet();
+        return 2;
+      } else {
+        // when the waitUnitFlushed() called the 2nd time, queue.size() will return 0
+        return 0;
+      }
+    });
+    index.waitUntilFlushed(MAX_WAIT);
+    verify(cache).getAsyncEventQueue(eq(expectedIndexName));
+  }
+
+}


[07/50] [abbrv] incubator-geode git commit: GEODE-17: pulse not started with gemfire security enabled

Posted by kl...@apache.org.
GEODE-17: pulse not started with gemfire security enabled


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

Branch: refs/heads/feature/GEODE-835
Commit: 97d61726e61c43bdada362b8b1ab867f954794dc
Parents: c4b9756
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri May 13 07:20:14 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri May 13 14:07:46 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/management/internal/ManagementAgent.java | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/97d61726/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
index adc69c4..af35c1b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/ManagementAgent.java
@@ -201,10 +201,9 @@ public class ManagementAgent {
         if (logger.isDebugEnabled()) {
           logger.debug(message);
         }
-
-        if (isCustomAuthorizer()){
-          System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
-        }
+      }
+      else if (isCustomAuthenticator()){
+        System.setProperty("spring.profiles.active", "pulse.authentication.gemfire");
       }
 
       // Find developer REST WAR file


[05/50] [abbrv] incubator-geode git commit: Fixing compile errors in eclipse

Posted by kl...@apache.org.
Fixing compile errors in eclipse

We were seeing compile errors in eclipse due to:
1) Missing gradle output dirs in the extensions module. I removed those
dirs from the eclipse classpath

2) Access restrictions on things like Unsafe. This was caused by a
change in gradle 2.12 to add the java 1.8 execution environment, rather
than the direct JDK dependency, to the project. That turned on access
restrictions in eclipse for  the execution environment.


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

Branch: refs/heads/feature/GEODE-835
Commit: c4b9756da6a549cd646fa797200382debf7a102c
Parents: 7908467
Author: Dan Smith <up...@apache.org>
Authored: Thu May 12 17:10:29 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri May 13 11:37:02 2016 -0700

----------------------------------------------------------------------
 extensions/geode-modules-tomcat7/build.gradle |  6 ++++++
 gradle/ide.gradle                             | 15 ++++++++++++---
 2 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b9756d/extensions/geode-modules-tomcat7/build.gradle
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-tomcat7/build.gradle b/extensions/geode-modules-tomcat7/build.gradle
index 975c976..3c75e56 100644
--- a/extensions/geode-modules-tomcat7/build.gradle
+++ b/extensions/geode-modules-tomcat7/build.gradle
@@ -49,4 +49,10 @@ dependencies {
   testCompile project(path: ':geode-junit')
   testCompile files(project(':geode-core').sourceSets.test.output)
   testCompile files(project(':extensions/geode-modules').sourceSets.test.output)
+
+  eclipse.classpath.file {
+    whenMerged { classpath ->
+      classpath.entries.removeAll { entry -> entry.path.contains('geode-modules/build')}
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b9756d/gradle/ide.gradle
----------------------------------------------------------------------
diff --git a/gradle/ide.gradle b/gradle/ide.gradle
index adaf8d9..03f6565 100644
--- a/gradle/ide.gradle
+++ b/gradle/ide.gradle
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import org.gradle.plugins.ide.eclipse.model.Container
 allprojects {
   apply plugin: 'idea'
   apply plugin: 'eclipse'
@@ -26,11 +27,19 @@ subprojects {
       downloadSources = true
       plusConfigurations += [ configurations.provided ]
       file {
-        // Remove the gradle output directories from the eclipse classpath.
-        // Unfortunately, using minusConfigurations does not work here, because
-        // it removes the entire geode-core project.
         whenMerged { classpath ->
+            // Remove the gradle output directories from the eclipse classpath.
+            // Unfortunately, using minusConfigurations does not work here, because
+            // it removes the entire geode-core project.
             classpath.entries.removeAll { entry -> entry.path.contains('geode-core/build')}
+
+            //By default, gradle adds the java 1.8 *execution environment*, which has access restrictions on
+            //things like Unsafe. Change it to a direct dependency on the workspace JDK
+            classpath.entries = classpath.entries.collect { entry -> 
+              entry.path.contains('org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE')
+                ? new Container('org.eclipse.jdt.launching.JRE_CONTAINER')
+                : entry
+            }
         }
       }
     }


[03/50] [abbrv] incubator-geode git commit: Revert "GEODE-1376: Cleaned up server port to be '0'."

Posted by kl...@apache.org.
Revert "GEODE-1376: Cleaned up server port to be '0'."

This reverts commit c33efb60630fe8bed7613c60f941810a7574f16b, which caused
a number of REST test failures in build 467 due to removal/rename of
test methods.

https://builds.apache.org/job/Geode-nightly/467/testReport/


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

Branch: refs/heads/feature/GEODE-835
Commit: d1a0748baf43c5cb5b2af46411d65d77778d1012
Parents: a881e9c
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri May 13 10:29:28 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri May 13 10:50:11 2016 -0700

----------------------------------------------------------------------
 ...stAPIOnRegionFunctionExecutionDUnitTest.java |  77 ++-
 .../web/controllers/RestAPITestBase.java        |  16 +-
 .../RestAPIsAndInterOpsDUnitTest.java           | 501 +++++++++++--------
 ...tAPIsOnGroupsFunctionExecutionDUnitTest.java |   6 +-
 ...APIsOnMembersFunctionExecutionDUnitTest.java |  19 +-
 .../controllers/RestAPIsWithSSLDUnitTest.java   | 495 +++++++++++-------
 .../internal/AutoConnectionSourceDUnitTest.java | 345 ++++++-------
 .../AutoConnectionSourceImplJUnitTest.java      |   5 +
 .../internal/LocatorLoadBalancingDUnitTest.java | 439 +++++++++-------
 .../cache/client/internal/LocatorTestBase.java  | 396 +++++++++------
 .../internal/cache/Bug47667DUnitTest.java       |  24 +-
 .../CacheServerManagementDUnitTest.java         | 260 ++++++----
 12 files changed, 1500 insertions(+), 1083 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIOnRegionFunctionExecutionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIOnRegionFunctionExecutionDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIOnRegionFunctionExecutionDUnitTest.java
index dc3986b..fedd119 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIOnRegionFunctionExecutionDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIOnRegionFunctionExecutionDUnitTest.java
@@ -36,7 +36,6 @@ import java.util.*;
 
 /**
  * Dunit Test to validate OnRegion function execution with REST APIs
- *
  * @since 8.0
  */
 
@@ -59,22 +58,22 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
       if (context instanceof RegionFunctionContext) {
         RegionFunctionContext rfContext = (RegionFunctionContext) context;
         rfContext.getDataSet().getCache().getLogger()
-            .info("Executing function :  SampleFunction.execute(hasResult=true) with filter: " + rfContext.getFilter() + "  " + rfContext);
+                .info("Executing function :  SampleFunction.execute(hasResult=true) with filter: " + rfContext.getFilter() + "  " + rfContext);
         if (rfContext.getArguments() instanceof Boolean) {
           /* return rfContext.getArguments(); */
           if (hasResult()) {
             rfContext.getResultSender().lastResult(
-                (Serializable) rfContext.getArguments());
+                    (Serializable) rfContext.getArguments());
           } else {
             rfContext
-                .getDataSet()
-                .getCache()
-                .getLogger()
-                .info(
-                    "Executing function :  SampleFunction.execute(hasResult=false) " + rfContext);
+                    .getDataSet()
+                    .getCache()
+                    .getLogger()
+                    .info(
+                            "Executing function :  SampleFunction.execute(hasResult=false) " + rfContext);
             while (true && !rfContext.getDataSet().isDestroyed()) {
               rfContext.getDataSet().getCache().getLogger()
-                  .info("For Bug43513 ");
+                      .info("For Bug43513 ");
               try {
                 Thread.sleep(100);
               } catch (InterruptedException ie) {
@@ -91,7 +90,7 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
               Thread.sleep(2000);
             } catch (InterruptedException e) {
               rfContext.getDataSet().getCache().getLogger()
-                  .warning("Got Exception : Thread Interrupted" + e);
+                      .warning("Got Exception : Thread Interrupted" + e);
             }
           }
           if (PartitionRegionHelper.isPartitionedRegion(rfContext.getDataSet())) {
@@ -101,11 +100,11 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
              * rfContext).get(key);
              */
             rfContext.getResultSender().lastResult(
-                (Serializable) PartitionRegionHelper.getLocalDataForContext(
-                    rfContext).get(key));
+                    (Serializable) PartitionRegionHelper.getLocalDataForContext(
+                            rfContext).get(key));
           } else {
             rfContext.getResultSender().lastResult(
-                (Serializable) rfContext.getDataSet().get(key));
+                    (Serializable) rfContext.getDataSet().get(key));
           }
           /* return (Serializable)rfContext.getDataSet().get(key); */
         } else if (rfContext.getArguments() instanceof Set) {
@@ -113,7 +112,7 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
           ArrayList vals = new ArrayList();
           for (Object key : origKeys) {
             Object val = PartitionRegionHelper
-                .getLocalDataForContext(rfContext).get(key);
+                    .getLocalDataForContext(rfContext).get(key);
             if (val != null) {
               vals.add(val);
             }
@@ -137,10 +136,10 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
           DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
           LogWriter logger = ds.getLogWriter();
           logger.info("Executing in SampleFunction on Server : "
-              + ds.getDistributedMember() + "with Context : " + context);
+                  + ds.getDistributedMember() + "with Context : " + context);
           while (ds.isConnected()) {
             logger
-                .fine("Just executing function in infinite loop for Bug43513");
+                    .fine("Just executing function in infinite loop for Bug43513");
             try {
               Thread.sleep(250);
             } catch (InterruptedException e) {
@@ -183,7 +182,7 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
 
   private boolean createPeerWithPR() {
     RegionAttributes ra = PartitionedRegionTestHelper.createRegionAttrsForPR(0,
-        10);
+            10);
     AttributesFactory raf = new AttributesFactory(ra);
     PartitionAttributesImpl pa = new PartitionAttributesImpl();
     pa.setAll(ra.getPartitionAttributes());
@@ -232,10 +231,10 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
   }
 
   private void createCacheAndRegisterFunction() {
-    restURLs.add(vm0.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm0.getHost().getHostName(), null)));
-    restURLs.add(vm1.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm1.getHost().getHostName(), null)));
-    restURLs.add(vm2.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm2.getHost().getHostName(), null)));
-    restURLs.add(vm3.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm3.getHost().getHostName(), null)));
+    restURLs.add(vm0.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm0, null)));
+    restURLs.add(vm1.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm1, null)));
+    restURLs.add(vm2.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm2, null)));
+    restURLs.add(vm3.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm3, null)));
 
     vm0.invoke("registerFunction(new SampleFunction())", () -> FunctionService.registerFunction(new SampleFunction()));
     vm1.invoke("registerFunction(new SampleFunction())", () -> FunctionService.registerFunction(new SampleFunction()));
@@ -284,7 +283,7 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
 
     createPeersWithPR(vm0, vm1, vm2, vm3);
 
-    vm3.invoke("populatePRRegion", () -> populatePRRegion());
+    vm3.invoke("populatePRRegion",() -> populatePRRegion());
 
     CloseableHttpResponse response = executeFunctionThroughRestCall("SampleFunction", PR_REGION_NAME, "key2", null, null, null);
     assertEquals(200, response.getStatusLine().getStatusCode());
@@ -306,16 +305,16 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
 
     createPeersWithPR(vm0, vm1, vm2, vm3);
 
-    vm3.invoke("populatePRRegion", () -> populatePRRegion());
+    vm3.invoke("populatePRRegion",() -> populatePRRegion());
 
     String jsonBody = "["
-        + "{\"@type\": \"double\",\"@value\": 210}"
-        + ",{\"@type\":\"com.gemstone.gemfire.rest.internal.web.controllers.Item\","
-        + "\"itemNo\":\"599\",\"description\":\"Part X Free on Bumper Offer\","
-        + "\"quantity\":\"2\","
-        + "\"unitprice\":\"5\","
-        + "\"totalprice\":\"10.00\"}"
-        + "]";
+            + "{\"@type\": \"double\",\"@value\": 210}"
+            + ",{\"@type\":\"com.gemstone.gemfire.rest.internal.web.controllers.Item\","
+            + "\"itemNo\":\"599\",\"description\":\"Part X Free on Bumper Offer\","
+            + "\"quantity\":\"2\","
+            + "\"unitprice\":\"5\","
+            + "\"totalprice\":\"10.00\"}"
+            + "]";
 
     CloseableHttpResponse response = executeFunctionThroughRestCall("SampleFunction", PR_REGION_NAME, null, jsonBody, null, null);
     assertEquals(200, response.getStatusLine().getStatusCode());
@@ -325,15 +324,15 @@ public class RestAPIOnRegionFunctionExecutionDUnitTest extends RestAPITestBase {
     assertCorrectInvocationCount(4, vm0, vm1, vm2, vm3);
 
     jsonBody = "["
-        + "{\"@type\": \"double\",\"@value\": 220}"
-        + ",{\"@type\":\"com.gemstone.gemfire.rest.internal.web.controllers.Item\","
-        + "\"itemNo\":\"609\",\"description\":\"Part X Free on Bumper Offer\","
-        + "\"quantity\":\"3\","
-        + "\"unitprice\":\"9\","
-        + "\"totalprice\":\"12.00\"}"
-        + "]";
-
-    resetInvocationCounts(vm0, vm1, vm2, vm3);
+            + "{\"@type\": \"double\",\"@value\": 220}"
+            + ",{\"@type\":\"com.gemstone.gemfire.rest.internal.web.controllers.Item\","
+            + "\"itemNo\":\"609\",\"description\":\"Part X Free on Bumper Offer\","
+            + "\"quantity\":\"3\","
+            + "\"unitprice\":\"9\","
+            + "\"totalprice\":\"12.00\"}"
+            + "]";
+
+    resetInvocationCounts(vm0,vm1,vm2,vm3);
 
     response = executeFunctionThroughRestCall("SampleFunction", PR_REGION_NAME, "key2", jsonBody, null, null);
     assertEquals(200, response.getStatusLine().getStatusCode());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPITestBase.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPITestBase.java b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPITestBase.java
index 9e8086c..fc71fd8 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPITestBase.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPITestBase.java
@@ -66,6 +66,7 @@ public class RestAPITestBase extends DistributedTestCase {
     if (agentUtil.findWarLocation("geode-web-api") == null) {
       fail("unable to locate geode-web-api WAR file");
     }
+    Wait.pause(1000); // TODO: replace this with Awaitility
     final Host host = Host.getHost(0);
     vm0 = host.getVM(0);
     vm1 = host.getVM(1);
@@ -101,10 +102,11 @@ public class RestAPITestBase extends DistributedTestCase {
     }
   }
 
-  public String createCacheWithGroups(final String hostName, final String groups) {
+  public String createCacheWithGroups(VM vm, final String groups) {
     RestAPITestBase test = new RestAPITestBase(getTestMethodName());
 
-    final int servicePort = AvailablePortHelper.getRandomAvailableTCPPort();
+    final String hostName = vm.getHost().getHostName();
+    final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
     Properties props = new Properties();
 
@@ -114,12 +116,12 @@ public class RestAPITestBase extends DistributedTestCase {
 
     props.setProperty(DistributionConfig.START_DEV_REST_API_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_BIND_ADDRESS_NAME, hostName);
-    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(servicePort));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(serverPort));
 
     InternalDistributedSystem ds = test.getSystem(props);
     cache = CacheFactory.create(ds);
 
-    String restEndPoint = "http://" + hostName + ":" + servicePort + "/gemfire-api/v1";
+    String restEndPoint = "http://" + hostName + ":" + serverPort + "/gemfire-api/v1";
     return restEndPoint;
   }
 
@@ -130,7 +132,7 @@ public class RestAPITestBase extends DistributedTestCase {
 
   protected CloseableHttpResponse executeFunctionThroughRestCall(String function, String regionName, String filter, String jsonBody, String groups,
                                                                  String members) {
-    System.out.println("Entering executeFunctionThroughRestCall");
+    LogWriterUtils.getLogWriter().info("Entering executeFunctionThroughRestCall");
     try {
       CloseableHttpClient httpclient = HttpClients.createDefault();
       Random randomGenerator = new Random();
@@ -138,7 +140,7 @@ public class RestAPITestBase extends DistributedTestCase {
 
       HttpPost post = createHTTPPost(function, regionName, filter, restURLIndex, groups, members, jsonBody);
 
-      System.out.println("Request: POST " + post.toString());
+      LogWriterUtils.getLogWriter().info("Request: POST " + post.toString());
       return httpclient.execute(post);
     } catch (Exception e) {
       throw new RuntimeException("unexpected exception", e);
@@ -183,7 +185,7 @@ public class RestAPITestBase extends DistributedTestCase {
     try {
       String httpResponseString = processHttpResponse(response);
       response.close();
-      System.out.println("Response : " + httpResponseString);
+      LogWriterUtils.getLogWriter().info("Response : " + httpResponseString);
       //verify function execution result
       JSONArray resultArray = new JSONArray(httpResponseString);
       assertEquals(resultArray.length(), expectedServerResponses);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
index 0f4b2d3..5b0eb17 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsAndInterOpsDUnitTest.java
@@ -16,7 +16,38 @@
  */
 package com.gemstone.gemfire.rest.internal.web.controllers;
 
-import com.gemstone.gemfire.cache.*;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.http.HttpEntity;
+import org.apache.http.client.ClientProtocolException;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpDelete;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionFactory;
@@ -33,31 +64,17 @@ import com.gemstone.gemfire.management.ManagementTestBase;
 import com.gemstone.gemfire.pdx.PdxInstance;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
-import org.apache.http.HttpEntity;
-import org.apache.http.client.ClientProtocolException;
-import org.apache.http.client.methods.*;
-import org.apache.http.entity.StringEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.util.*;
 
 /**
  * Dunit Test containing inter - operations between REST Client and Gemfire cache client
- *
  * @since 8.0
  */
 
 public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
-
+  
   private static final long serialVersionUID = -254776154266339226L;
 
   private ManagementTestBase helper;
@@ -72,7 +89,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
 
   private static final String[] PARAM_QUERY_IDS_ARRAY = { "findAllPeople",
       "filterByGender", "filterByLastName" };
-
+  
   final static String QUERY_ARGS = "["
       + "{"
       + "\"@type\": \"string\","
@@ -81,29 +98,29 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
       + "]";
 
   final static String PERSON_AS_JSON_CAS = "{"
-      + "\"@old\" :"
+      + "\"@old\" :" 
       + "{"
       + "\"@type\": \"com.gemstone.gemfire.rest.internal.web.controllers.Person\","
       + "\"id\": 101," + " \"firstName\": \"Mithali\","
       + " \"middleName\": \"Dorai\"," + " \"lastName\": \"Raj\","
-      + " \"birthDate\": \"12/04/1982\"," + "\"gender\": \"FEMALE\""
+      + " \"birthDate\": \"12/04/1982\"," + "\"gender\": \"FEMALE\"" 
       + "},"
-      + "\"@new\" :"
+      + "\"@new\" :" 
       + "{"
       + "\"@type\": \"com.gemstone.gemfire.rest.internal.web.controllers.Person\","
       + "\"id\": 1101," + " \"firstName\": \"Virat\","
       + " \"middleName\": \"Premkumar\"," + " \"lastName\": \"Kohli\","
-      + " \"birthDate\": \"08/11/1988\"," + "\"gender\": \"MALE\""
+      + " \"birthDate\": \"08/11/1988\"," + "\"gender\": \"MALE\"" 
       + "}"
       + "}";
-
+    
   final static String PERSON_AS_JSON_REPLACE = "{"
       + "\"@type\": \"com.gemstone.gemfire.rest.internal.web.controllers.Person\","
       + "\"id\": 501," + " \"firstName\": \"Barack\","
       + " \"middleName\": \"Hussein\"," + " \"lastName\": \"Obama\","
-      + " \"birthDate\": \"04/08/1961\"," + "\"gender\": \"MALE\""
+      + " \"birthDate\": \"04/08/1961\"," + "\"gender\": \"MALE\"" 
       + "}";
-
+  
   private static final String PERSON_LIST_AS_JSON = "[" + "{"
       + "\"@type\": \"com.gemstone.gemfire.rest.internal.web.controllers.Person\","
       + "\"id\": 3," + " \"firstName\": \"Nishka3\","
@@ -161,38 +178,86 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
   protected final void postTearDownLocatorTestBase() throws Exception {
     disconnectAllFromDS();
   }
-
-  public void doPutsInClientCache() {
+  
+  public static String startBridgeServerWithRestServiceOnInVM(VM vm, final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe) {
+    
+    final String hostName = vm.getHost().getHostName(); 
+    final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    
+    //create Cache of given VM and start HTTP service with REST APIs service
+    new RestAPIsAndInterOpsDUnitTest("temp").startBridgeServer(hostName, serverPort, groups, locators, regions, probe);
+   
+    String restEndPoint =  "http://" + hostName + ":" + serverPort + "/gemfire-api/v1";
+    return restEndPoint;
+  }
+  
+  @SuppressWarnings("deprecation")
+  protected int startBridgeServer(String hostName, int restServicerPort, final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe) {
+            
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+    props.setProperty(DistributionConfig.START_DEV_REST_API_NAME, "true");
+    props.setProperty(DistributionConfig.HTTP_SERVICE_BIND_ADDRESS_NAME, hostName);
+    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(restServicerPort));
+    
+    DistributedSystem ds = getSystem(props);
+    Cache cache = CacheFactory.create(ds);
+    ((GemFireCacheImpl)cache).setReadSerialized(true);
+    AttributesFactory factory = new AttributesFactory();
+    
+    factory.setEnableBridgeConflation(true);
+    factory.setDataPolicy(DataPolicy.REPLICATE);
+    RegionAttributes attrs = factory.create();
+    for(int i = 0; i < regions.length; i++) {
+      cache.createRegion(regions[i], attrs);
+    }
+    
+    CacheServer server = cache.addCacheServer();
+    final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    server.setPort(serverPort);
+    server.setGroups(groups);
+    server.setLoadProbe(probe);
+    try {
+      server.start();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    remoteObjects.put(CACHE_KEY, cache);
+    return new Integer(serverPort);
+  }
+  
+  public static void doPutsInClientCache() {
     ClientCache cache = GemFireCacheImpl.getInstance();
     assertNotNull(cache);
     Region<String, Object> region = cache.getRegion(PEOPLE_REGION_NAME);
-
+   
     //put person object
     final Person person1 = new Person(101L, "Mithali", "Dorai", "Raj", DateTimeUtils.createDate(1982, Calendar.DECEMBER, 4), Gender.FEMALE);
     final Person person2 = new Person(102L, "Sachin", "Ramesh", "Tendulkar", DateTimeUtils.createDate(1975, Calendar.DECEMBER, 14), Gender.MALE);
     final Person person3 = new Person(103L, "Saurabh", "Baburav", "Ganguly", DateTimeUtils.createDate(1972, Calendar.AUGUST, 29), Gender.MALE);
     final Person person4 = new Person(104L, "Rahul", "subrymanyam", "Dravid", DateTimeUtils.createDate(1979, Calendar.MARCH, 17), Gender.MALE);
     final Person person5 = new Person(105L, "Jhulan", "Chidambaram", "Goswami", DateTimeUtils.createDate(1983, Calendar.NOVEMBER, 25), Gender.FEMALE);
-
+   
     region.put("1", person1);
     region.put("2", person2);
     region.put("3", person3);
     region.put("4", person4);
     region.put("5", person5);
-
+    
     final Person person6 = new Person(101L, "Rahul", "Rajiv", "Gndhi", DateTimeUtils.createDate(1970, Calendar.MAY, 14), Gender.MALE);
     final Person person7 = new Person(102L, "Narendra", "Damodar", "Modi", DateTimeUtils.createDate(1945, Calendar.DECEMBER, 24), Gender.MALE);
     final Person person8 = new Person(103L, "Atal", "Bihari", "Vajpayee", DateTimeUtils.createDate(1920, Calendar.AUGUST, 9), Gender.MALE);
     final Person person9 = new Person(104L, "Soniya", "Rajiv", "Gandhi", DateTimeUtils.createDate(1929, Calendar.MARCH, 27), Gender.FEMALE);
     final Person person10 = new Person(104L, "Priyanka", "Robert", "Gandhi", DateTimeUtils.createDate(1973, Calendar.APRIL, 15), Gender.FEMALE);
-
+    
     final Person person11 = new Person(104L, "Murali", "Manohar", "Joshi", DateTimeUtils.createDate(1923, Calendar.APRIL, 25), Gender.MALE);
     final Person person12 = new Person(104L, "Lalkrishna", "Parmhansh", "Advani", DateTimeUtils.createDate(1910, Calendar.JANUARY, 01), Gender.MALE);
     final Person person13 = new Person(104L, "Shushma", "kumari", "Swaraj", DateTimeUtils.createDate(1943, Calendar.AUGUST, 10), Gender.FEMALE);
     final Person person14 = new Person(104L, "Arun", "raman", "jetly", DateTimeUtils.createDate(1942, Calendar.OCTOBER, 27), Gender.MALE);
     final Person person15 = new Person(104L, "Amit", "kumar", "shah", DateTimeUtils.createDate(1958, Calendar.DECEMBER, 21), Gender.MALE);
     final Person person16 = new Person(104L, "Shila", "kumari", "Dixit", DateTimeUtils.createDate(1927, Calendar.FEBRUARY, 15), Gender.FEMALE);
-
+    
     Map<String, Object> userMap = new HashMap<String, Object>();
     userMap.put("6", person6);
     userMap.put("7", person7);
@@ -205,15 +270,15 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
     userMap.put("14", person14);
     userMap.put("15", person15);
     userMap.put("16", person16);
-
+    
     region.putAll(userMap);
-
+    
     if (cache != null)
       cache.getLogger().info("Gemfire Cache Client: Puts successfully done");
-
+    
   }
 
-  public void doQueryOpsUsingRestApis(String restEndpoint) {
+  public static void doQueryOpsUsingRestApis(String restEndpoint) {
     String currentQueryOp = null;
     try {
       // Query TestCase-1 :: Prepare parameterized Queries
@@ -226,7 +291,8 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         CloseableHttpResponse createNamedQueryResponse = httpclient.execute(post);
         assertEquals(createNamedQueryResponse.getStatusLine().getStatusCode(), 201);
         assertNotNull(createNamedQueryResponse.getEntity());
-        createNamedQueryResponse.close();
+        createNamedQueryResponse.close();        
+        
 
         post = new HttpPost(restEndpoint + findPeopleByGenderQuery);
         post.addHeader("Content-Type", "application/json");
@@ -235,6 +301,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         assertEquals(createNamedQueryResponse.getStatusLine().getStatusCode(), 201);
         assertNotNull(createNamedQueryResponse.getEntity());
         createNamedQueryResponse.close();
+        
 
         post = new HttpPost(restEndpoint + findPeopleByLastNameQuery);
         post.addHeader("Content-Type", "application/json");
@@ -244,7 +311,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         assertNotNull(createNamedQueryResponse.getEntity());
         createNamedQueryResponse.close();
       }
-
+      
       // Query TestCase-2 :: List all parameterized queries
       {
         currentQueryOp = "listAllQueries";
@@ -253,7 +320,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         CloseableHttpResponse listAllQueriesResponse = httpclient.execute(get);
         assertEquals(listAllQueriesResponse.getStatusLine().getStatusCode(), 200);
         assertNotNull(listAllQueriesResponse.getEntity());
-
+        
         HttpEntity entity = listAllQueriesResponse.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -264,19 +331,19 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
           sb.append(line);
         }
         listAllQueriesResponse.close();
-
+      
         // Check whether received response contains expected query IDs.
 
         JSONObject jsonObject = new JSONObject(sb.toString());
         JSONArray jsonArray = jsonObject.getJSONArray("queries");
         for (int i = 0; i < jsonArray.length(); i++) {
           assertTrue(
-              "PREPARE_PARAMETERIZED_QUERY: function IDs are not matched",
-              Arrays.asList(PARAM_QUERY_IDS_ARRAY).contains(
-                  jsonArray.getJSONObject(i).getString("id")));
+            "PREPARE_PARAMETERIZED_QUERY: function IDs are not matched",
+            Arrays.asList(PARAM_QUERY_IDS_ARRAY).contains(
+                jsonArray.getJSONObject(i).getString("id")));
         }
-      }
-
+      }  
+      
       // Query TestCase-3 :: Run the specified named query passing in scalar values for query parameters.
       {
         currentQueryOp = "filterByLastName";
@@ -284,35 +351,41 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         HttpPost post = new HttpPost(restEndpoint + "/queries/filterByLastName");
         post.addHeader("Content-Type", "application/json");
         post.addHeader("Accept", "application/json");
-        StringEntity entity = new StringEntity(QUERY_ARGS);
+        StringEntity entity = new StringEntity(QUERY_ARGS);       
         post.setEntity(entity);
         CloseableHttpResponse runNamedQueryResponse = httpclient.execute(post);
 
         assertEquals(200, runNamedQueryResponse.getStatusLine().getStatusCode());
         assertNotNull(runNamedQueryResponse.getEntity());
       }
-    } catch (Exception e) {
+    } catch ( Exception e ) {
       throw new RuntimeException("unexpected exception", e);
     }
   }
-
-  public void verifyUpdatesInClientCache() {
+  
+  public static void verifyUpdatesInClientCache() {
     ClientCache cache = GemFireCacheImpl.getInstance();
     assertNotNull(cache);
     Region<String, Object> region = cache.getRegion(PEOPLE_REGION_NAME);
-
+    
     {
-      Person expectedPerson = new Person(3L, "Nishka3", "Nilkanth3", "Patel3", DateTimeUtils.createDate(2009, Calendar.JULY, 31), Gender.FEMALE);
+      Person expectedPerson = new Person(3L, "Nishka3", "Nilkanth3", "Patel3", DateTimeUtils.createDate(2009, Calendar.JULY, 31), Gender.FEMALE );
       Object value = region.get("3");
       if (value instanceof PdxInstance) {
-        PdxInstance pi3 = (PdxInstance) value;
+        PdxInstance pi3  = (PdxInstance) value;
         Person actualPerson = (Person) pi3.getObject();
-        comparePersonObject(expectedPerson, actualPerson);
+        assertEquals(actualPerson.getId(), expectedPerson.getId());
+        assertEquals(actualPerson.getFirstName(), expectedPerson.getFirstName());
+        assertEquals(actualPerson.getMiddleName(), expectedPerson.getMiddleName());
+        assertEquals(actualPerson.getLastName(), expectedPerson.getLastName());
+        assertEquals(actualPerson.getBirthDate(), expectedPerson.getBirthDate());
+        assertEquals(actualPerson.getGender(), expectedPerson.getGender());
       } else if (value instanceof Person) {
         fail("VerifyUpdatesInClientCache, Get on key 3, Expected to get value of type PdxInstance ");
       }
     }
-
+     
+    
     //TODO: uncomment it once following issue encountered in put?op=CAS is fixed or document the issue
     // CAS functionality is not working in following test case
     // step-1: Java client, Region.put("K", A);
@@ -331,47 +404,43 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
       }
     }
     */
-
+    
     //verify update on key "2"
     {
-      Person expectedPerson = new Person(501L, "Barack", "Hussein", "Obama", DateTimeUtils.createDate(1961, Calendar.APRIL, 8), Gender.MALE);
+      Person expectedPerson = new Person(501L, "Barack", "Hussein", "Obama", DateTimeUtils.createDate(1961, Calendar.APRIL, 8), Gender.MALE );
       Object value = region.get("2");
       if (value instanceof PdxInstance) {
-        PdxInstance pi3 = (PdxInstance) value;
+        PdxInstance pi3  = (PdxInstance) value;
         Person actualPerson = (Person) pi3.getObject();
-        comparePersonObject(expectedPerson, actualPerson);
-      } else {
+        assertEquals(actualPerson.getId(), expectedPerson.getId());
+        assertEquals(actualPerson.getFirstName(), expectedPerson.getFirstName());
+        assertEquals(actualPerson.getMiddleName(), expectedPerson.getMiddleName());
+        assertEquals(actualPerson.getLastName(), expectedPerson.getLastName());
+        assertEquals(actualPerson.getBirthDate(), expectedPerson.getBirthDate());
+        assertEquals(actualPerson.getGender(), expectedPerson.getGender());
+      }else {
         fail("VerifyUpdatesInClientCache, Get on key 2, Expected to get value of type PdxInstance ");
       }
     }
-
+    
     //verify Deleted key "13"
     {
       Object obj = region.get("13");
       assertEquals(obj, null);
-
+      
       obj = region.get("14");
       assertEquals(obj, null);
-
+      
       obj = region.get("15");
       assertEquals(obj, null);
-
+      
       obj = region.get("16");
       assertEquals(obj, null);
     }
-
+    
   }
-
-  private void comparePersonObject(Person expectedPerson, Person actualPerson) {
-    assertEquals(actualPerson.getId(), expectedPerson.getId());
-    assertEquals(actualPerson.getFirstName(), expectedPerson.getFirstName());
-    assertEquals(actualPerson.getMiddleName(), expectedPerson.getMiddleName());
-    assertEquals(actualPerson.getLastName(), expectedPerson.getLastName());
-    assertEquals(actualPerson.getBirthDate(), expectedPerson.getBirthDate());
-    assertEquals(actualPerson.getGender(), expectedPerson.getGender());
-  }
-
-  public void doUpdatesUsingRestApis(String restEndpoint) {
+  
+  public static void doUpdatesUsingRestApis(String restEndpoint) {
     //UPdate keys using REST calls
     {
 
@@ -388,7 +457,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         throw new RuntimeException("unexpected exception", e);
       }
     }
-
+    
     //Delete Single keys
     {
       try {
@@ -401,7 +470,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         throw new RuntimeException("unexpected exception", e);
       }
     }
-
+    
     //Delete set of keys
     {
       try {
@@ -414,7 +483,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         throw new RuntimeException("unexpected exception", e);
       }
     }
-
+    
     //REST put?op=CAS for key 1
     /*
     try {   
@@ -430,15 +499,15 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
     }catch (HttpServerErrorException se) {
       fail("Caught HttpServerErrorException while doing put with op=cas");
     }
-    */
-
+    */ 
+    
     //REST put?op=REPLACE for key 2
     {
       /*HttpEntity<Object> entity = new HttpEntity<Object>(PERSON_AS_JSON_REPLACE, headers);
       ResponseEntity<String> result = RestTestUtils.getRestTemplate().exchange(
         restEndpoint + "/People/2?op=replace",
       HttpMethod.PUT, entity, String.class);*/
-
+      
       try {
         CloseableHttpClient httpclient = HttpClients.createDefault();
         HttpPut put = new HttpPut(restEndpoint
@@ -453,14 +522,14 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
       }
     }
   }
-
-  public void fetchRestServerEndpoints(String restEndpoint) {
+  
+  public static void fetchRestServerEndpoints(String restEndpoint) {
     HttpGet get = new HttpGet(restEndpoint + "/servers");
     get.addHeader("Content-Type", "application/json");
     get.addHeader("Accept", "application/json");
     CloseableHttpClient httpclient = HttpClients.createDefault();
     CloseableHttpResponse response;
-
+    
     try {
       response = httpclient.execute(get);
       HttpEntity entity = response.getEntity();
@@ -472,18 +541,18 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
       while ((line = reader.readLine()) != null) {
         str.append(line);
       }
-
+      
       //validate the satus code
       assertEquals(response.getStatusLine().getStatusCode(), 200);
-
-      if (response.getStatusLine().getStatusCode() == 200) {
+      
+      if(response.getStatusLine().getStatusCode() == 200) {
         JSONArray jsonArray = new JSONArray(str.toString());
-
+        
         //verify total number of REST service endpoints in DS
         assertEquals(jsonArray.length(), 2);
       }
-
-    } catch (ClientProtocolException e) {
+      
+    } catch (ClientProtocolException e) { 
       e.printStackTrace();
       fail(" Rest Request should not have thrown ClientProtocolException!");
     } catch (IOException e) {
@@ -493,26 +562,26 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
       e.printStackTrace();
       fail(" Rest Request should not have thrown  JSONException!");
     }
-
+    
   }
-
-  public void doGetsUsingRestApis(String restEndpoint) {
-
+  
+  public static void doGetsUsingRestApis(String restEndpoint) {
+    
     //HttpHeaders headers = setAcceptAndContentTypeHeaders(); 
     String currentOperation = null;
     JSONObject jObject;
     JSONArray jArray;
-    try {
+    try {    
       //1. Get on key="1" and validate result.
       {
         currentOperation = "GET on key 1";
-
+        
         HttpGet get = new HttpGet(restEndpoint + "/People/1");
         get.addHeader("Content-Type", "application/json");
         get.addHeader("Accept", "application/json");
         CloseableHttpClient httpclient = HttpClients.createDefault();
         CloseableHttpResponse response = httpclient.execute(get);
-
+        
         HttpEntity entity = response.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -522,26 +591,27 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         while ((line = reader.readLine()) != null) {
           str.append(line);
         }
-
+        
         jObject = new JSONObject(str.toString());
-
+      
         assertEquals(jObject.get("id"), 101);
         assertEquals(jObject.get("firstName"), "Mithali");
         assertEquals(jObject.get("middleName"), "Dorai");
         assertEquals(jObject.get("lastName"), "Raj");
         assertEquals(jObject.get("gender"), Gender.FEMALE.name());
       }
-
+      
       //2. Get on key="16" and validate result.
       {
         currentOperation = "GET on key 16";
 
+        
         HttpGet get = new HttpGet(restEndpoint + "/People/16");
         get.addHeader("Content-Type", "application/json");
         get.addHeader("Accept", "application/json");
         CloseableHttpClient httpclient = HttpClients.createDefault();
         CloseableHttpResponse response = httpclient.execute(get);
-
+        
         HttpEntity entity = response.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -551,16 +621,17 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         while ((line = reader.readLine()) != null) {
           str.append(line);
         }
-
+        
         jObject = new JSONObject(str.toString());
-
+        
+      
         assertEquals(jObject.get("id"), 104);
         assertEquals(jObject.get("firstName"), "Shila");
         assertEquals(jObject.get("middleName"), "kumari");
         assertEquals(jObject.get("lastName"), "Dixit");
         assertEquals(jObject.get("gender"), Gender.FEMALE.name());
       }
-
+      
       //3. Get all (getAll) entries in Region
       {
 
@@ -571,7 +642,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         CloseableHttpResponse result = httpclient.execute(get);
         assertEquals(result.getStatusLine().getStatusCode(), 200);
         assertNotNull(result.getEntity());
-
+      
         HttpEntity entity = result.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -582,16 +653,16 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
           sb.append(line);
         }
         result.close();
-
+        
         try {
           jObject = new JSONObject(sb.toString());
           jArray = jObject.getJSONArray("People");
           assertEquals(jArray.length(), 16);
         } catch (JSONException e) {
-          fail(" Rest Request ::" + currentOperation + " :: should not have thrown JSONException ");
+          fail(" Rest Request ::" + currentOperation +  " :: should not have thrown JSONException ");
         }
       }
-
+      
       //4. GetAll?limit=10 (10 entries) and verify results
       {
         HttpGet get = new HttpGet(restEndpoint + "/People?limit=10");
@@ -601,7 +672,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         CloseableHttpResponse response = httpclient.execute(get);
         assertEquals(response.getStatusLine().getStatusCode(), 200);
         assertNotNull(response.getEntity());
-
+        
         HttpEntity entity = response.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -611,19 +682,19 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         while ((line = reader.readLine()) != null) {
           str.append(line);
         }
-
+        
         try {
           jObject = new JSONObject(str.toString());
           jArray = jObject.getJSONArray("People");
           assertEquals(jArray.length(), 10);
         } catch (JSONException e) {
-          fail(" Rest Request ::" + currentOperation + " :: should not have thrown JSONException ");
+          fail(" Rest Request ::" + currentOperation +  " :: should not have thrown JSONException ");
         }
       }
-
+      
       //5. Get keys - List all keys in region
-      {
-
+      {  
+        
         HttpGet get = new HttpGet(restEndpoint + "/People/keys");
         get.addHeader("Content-Type", "application/json");
         get.addHeader("Accept", "application/json");
@@ -631,7 +702,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         CloseableHttpResponse response = httpclient.execute(get);
         assertEquals(response.getStatusLine().getStatusCode(), 200);
         assertNotNull(response.getEntity());
-
+        
         HttpEntity entity = response.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -641,19 +712,19 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         while ((line = reader.readLine()) != null) {
           str.append(line);
         }
-
+        
         try {
           jObject = new JSONObject(str.toString());
-          jArray = jObject.getJSONArray("keys");
+          jArray = jObject.getJSONArray("keys"); 
           assertEquals(jArray.length(), 16);
         } catch (JSONException e) {
-          fail(" Rest Request ::" + currentOperation + " :: should not have thrown JSONException ");
+          fail(" Rest Request ::" + currentOperation +  " :: should not have thrown JSONException ");
         }
-      }
-
+      } 
+      
       //6. Get data for specific keys
-      {
-
+      {  
+       
         HttpGet get = new HttpGet(restEndpoint + "/People/1,3,5,7,9,11");
         get.addHeader("Content-Type", "application/json");
         get.addHeader("Accept", "application/json");
@@ -661,7 +732,7 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         CloseableHttpResponse response = httpclient.execute(get);
         assertEquals(response.getStatusLine().getStatusCode(), 200);
         assertNotNull(response.getEntity());
-
+        
         HttpEntity entity = response.getEntity();
         InputStream content = entity.getContent();
         BufferedReader reader = new BufferedReader(new InputStreamReader(
@@ -671,44 +742,53 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
         while ((line = reader.readLine()) != null) {
           str.append(line);
         }
-
+        
         try {
           jObject = new JSONObject(str.toString());
           jArray = jObject.getJSONArray("People");
           assertEquals(jArray.length(), 6);
-
+          
         } catch (JSONException e) {
-          fail(" Rest Request ::" + currentOperation + " :: should not have thrown JSONException ");
+          fail(" Rest Request ::" + currentOperation +  " :: should not have thrown JSONException ");
         }
-      }
-    } catch (Exception e) {
-      throw new RuntimeException("unexpected exception", e);
+      } 
+    }catch ( Exception e ) {
+      throw new RuntimeException("unexpected exception", e); 
     }
   }
 
-  public void createRegionInClientCache() {
+  public static void createRegionInClientCache() {
     ClientCache cache = GemFireCacheImpl.getInstance();
     assertNotNull(cache);
     ClientRegionFactory<String, Object> crf = cache
         .createClientRegionFactory(ClientRegionShortcut.PROXY);
-    crf.create(PEOPLE_REGION_NAME);
+    Region<String, Object> region = crf.create(PEOPLE_REGION_NAME);
+
+  }
+
+  public static void createRegionInManager() {
+    Cache cache = GemFireCacheImpl.getInstance();
+    assertNotNull(cache);
+
+    RegionFactory<String, Object> rf = cache
+        .createRegionFactory(RegionShortcut.REPLICATE);
+    Region<String, Object> region = rf.create(PEOPLE_REGION_NAME);
   }
 
-  public void createRegion() {
+  public static void createRegionInPeerServer() {
     Cache cache = GemFireCacheImpl.getInstance();
     assertNotNull(cache);
 
-    RegionFactory<String, Object> regionFactory = cache
+    RegionFactory<String, Object> rf = cache
         .createRegionFactory(RegionShortcut.REPLICATE);
-    regionFactory.create(PEOPLE_REGION_NAME);
+    Region<String, Object> region = rf.create(PEOPLE_REGION_NAME);
   }
 
   /**
-   * InterOps Test between REST-client, Peer Cache Client and Client Cache
-   *
+   * InterOps Test between REST-client, Peer Cache Client and Client Cache 
    * @throws Exception
    */
-
+ 
   public void testInterOpsWithReplicatedRegion() throws Exception {
 
     final Host host = Host.getHost(0);
@@ -719,46 +799,47 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
 
     // start locator
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
-    locator.invoke("Start Locator", () -> startLocator(locator.getHost(), locatorPort, ""));
-
+    
+    startLocatorInVM(locator, locatorPort, "");
+    
     // find locators
     String locators = NetworkUtils.getServerHostName(locator.getHost()) + "[" + locatorPort
         + "]";
 
     // start manager (peer cache)
-    manager.invoke("Start manager", () -> startManager(null, locators,
-        new String[] { REGION_NAME }, CacheServer.DEFAULT_LOAD_PROBE));
-
+    int managerPort = startManagerInVM(manager,/* groups */null, locators,
+        new String[] {REGION_NAME}, CacheServer.DEFAULT_LOAD_PROBE);
+    
     //start startCacheServer With RestService enabled
-    String restEndpoint = (String) server.invoke(RestAPIsAndInterOpsDUnitTest.class,
-        "startBridgeServerWithRestServiceOnInVM", new Object[] { server, null, locators, new String[] { REGION_NAME }, CacheServer.DEFAULT_LOAD_PROBE });
-
+    String restEndpoint = (String)server.invoke(RestAPIsAndInterOpsDUnitTest.class,
+        "startBridgeServerWithRestServiceOnInVM", new Object[] { server ,  null, locators, new String[] {REGION_NAME}, CacheServer.DEFAULT_LOAD_PROBE });
+    
     // create a client cache
-    client.invoke("Create Client", () -> createClientCache(NetworkUtils.getServerHostName(locator.getHost()), locatorPort));
-
+    createClientCacheInVM(client, NetworkUtils.getServerHostName(locator.getHost()),
+        locatorPort);
+    
     // create region in Manager, peer cache and Client cache nodes
-    manager.invoke("create region", () -> createRegion());
-    server.invoke("create region", () -> createRegion());
-    client.invoke("create region", () -> createRegionInClientCache());
-
+    manager.invoke(() -> RestAPIsAndInterOpsDUnitTest.createRegionInManager());
+    server.invoke(() -> RestAPIsAndInterOpsDUnitTest.createRegionInPeerServer());
+    client.invoke(() -> RestAPIsAndInterOpsDUnitTest.createRegionInClientCache());
+    
     // do some person puts from clientcache
-    client.invoke("doPutsInClientCache", () -> doPutsInClientCache());
+    client.invoke(() -> RestAPIsAndInterOpsDUnitTest.doPutsInClientCache());
 
     //TEST: fetch all available REST endpoints
-    fetchRestServerEndpoints(restEndpoint);
-
+    RestAPIsAndInterOpsDUnitTest.fetchRestServerEndpoints(restEndpoint);
+   
     // Controller VM - config REST Client and make HTTP calls
-    doGetsUsingRestApis(restEndpoint);
-
+    RestAPIsAndInterOpsDUnitTest.doGetsUsingRestApis(restEndpoint);
+    
     //update Data using REST APIs
-    doUpdatesUsingRestApis(restEndpoint);
-
-    client.invoke("verifyUpdatesInClientCache", () -> verifyUpdatesInClientCache());
+    RestAPIsAndInterOpsDUnitTest.doUpdatesUsingRestApis(restEndpoint);
+    
+    client.invoke(() -> RestAPIsAndInterOpsDUnitTest.verifyUpdatesInClientCache());
 
     //Querying
-    doQueryOpsUsingRestApis(restEndpoint);
-
+    RestAPIsAndInterOpsDUnitTest.doQueryOpsUsingRestApis(restEndpoint);
+    
     // stop the client and make sure the bridge server notifies
     // stopBridgeMemberVM(client);
     helper.closeCache(locator);
@@ -768,44 +849,66 @@ public class RestAPIsAndInterOpsDUnitTest extends LocatorTestBase {
 
   }
 
-  private void createClientCache(final String host, final int port) throws Exception {
-    // Connect using the GemFire locator and create a Caching_Proxy cache
-    ClientCache cache = new ClientCacheFactory().setPdxReadSerialized(true).addPoolLocator(host, port).create();
-    cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
-  }
-
-  private int startManager(final String[] groups,
-      final String locators, final String[] regions, final ServerLoadProbe probe) throws IOException {
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
-    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
-
-    props.setProperty("jmx-manager", "true");
-    props.setProperty("jmx-manager-start", "true");
-    props.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, "0");
-
-    final int httpPort = AvailablePortHelper.getRandomAvailableTCPPort();
-    //Set REST service related configuration
-    props.setProperty(DistributionConfig.START_DEV_REST_API_NAME, "true");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_BIND_ADDRESS_NAME, "localhost");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
-
-    DistributedSystem ds = getSystem(props);
-    Cache cache = CacheFactory.create(ds);
-    AttributesFactory factory = new AttributesFactory();
+  private void createClientCacheInVM(VM vm, final String host, final int port) throws Exception {
+    SerializableRunnable connect = new SerializableRunnable(
+        "Start Cache client") {
+      public void run() {
+        // Connect using the GemFire locator and create a Caching_Proxy cache
+        ClientCache c = new ClientCacheFactory().setPdxReadSerialized(true).addPoolLocator(host, port)
+            .create();
+       
+        Region r = c.createClientRegionFactory(
+            ClientRegionShortcut.PROXY).create(REGION_NAME);
+      }
+    };
 
-    factory.setEnableBridgeConflation(true);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attrs = factory.create();
-    for (int i = 0; i < regions.length; i++) {
-      cache.createRegion(regions[i], attrs);
+    if (vm == null) {
+      connect.run();
+    } else {
+      vm.invoke(connect);
     }
-    CacheServer server = cache.addCacheServer();
-    server.setPort(0);
-    server.setGroups(groups);
-    server.setLoadProbe(probe);
-    server.start();
+  }
 
-    return new Integer(server.getPort());
+  private int startManagerInVM(VM vm, final String[] groups,
+      final String locators, final String[] regions, final ServerLoadProbe probe) {
+    SerializableCallable connect = new SerializableCallable("Start Manager ") {
+      public Object call() throws IOException {
+        Properties props = new Properties();
+        props
+            .setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+        props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+
+        props.setProperty("jmx-manager", "true");
+        props.setProperty("jmx-manager-start", "true");
+        props.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, "0");
+        
+        final int httpPort = AvailablePortHelper.getRandomAvailableTCPPort();
+        //Set REST service related configuration
+        props.setProperty(DistributionConfig.START_DEV_REST_API_NAME, "true");
+        props.setProperty(DistributionConfig.HTTP_SERVICE_BIND_ADDRESS_NAME, "localhost");
+        props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(httpPort));
+
+        DistributedSystem ds = getSystem(props);
+        Cache cache = CacheFactory.create(ds);
+        AttributesFactory factory = new AttributesFactory();
+        
+        factory.setEnableBridgeConflation(true);
+        factory.setDataPolicy(DataPolicy.REPLICATE);
+        RegionAttributes attrs = factory.create();
+        for (int i = 0; i < regions.length; i++) {
+          cache.createRegion(regions[i], attrs);
+        }
+        CacheServer server = cache.addCacheServer();
+        final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+        server.setPort(serverPort);
+        server.setGroups(groups);
+        server.setLoadProbe(probe);
+        server.start();
+        
+        return new Integer(serverPort);
+      }
+    };
+    Integer port = (Integer) vm.invoke(connect);
+    return port.intValue();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnGroupsFunctionExecutionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnGroupsFunctionExecutionDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnGroupsFunctionExecutionDUnitTest.java
index a567700..61e0925 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnGroupsFunctionExecutionDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnGroupsFunctionExecutionDUnitTest.java
@@ -39,9 +39,9 @@ public class RestAPIsOnGroupsFunctionExecutionDUnitTest extends RestAPITestBase
   }
 
   private void setupCacheWithGroupsAndFunction() {
-    restURLs.add(vm0.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm0.getHost().getHostName(), "g0,gm")));
-    restURLs.add(vm1.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm1.getHost().getHostName(), "g1")));
-    restURLs.add(vm2.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm2.getHost().getHostName(), "g0,g1")));
+    restURLs.add(vm0.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm0, "g0,gm")));
+    restURLs.add(vm1.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm1, "g1")));
+    restURLs.add(vm2.invoke("createCacheWithGroups", () -> createCacheWithGroups(vm2, "g0,g1")));
 
     vm0.invoke("registerFunction(new OnGroupsFunction())", () -> FunctionService.registerFunction(new OnGroupsFunction()));
     vm1.invoke("registerFunction(new OnGroupsFunction())", () -> FunctionService.registerFunction(new OnGroupsFunction()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnMembersFunctionExecutionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnMembersFunctionExecutionDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnMembersFunctionExecutionDUnitTest.java
index fd6b312..ca01830 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnMembersFunctionExecutionDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsOnMembersFunctionExecutionDUnitTest.java
@@ -44,7 +44,7 @@ public class RestAPIsOnMembersFunctionExecutionDUnitTest extends RestAPITestBase
     @Override
     public void execute(FunctionContext context) {
 
-      System.out.println("SWAP:1:executing OnMembersFunction:" + invocationCount);
+      LogWriterUtils.getLogWriter().fine("SWAP:1:executing OnMembersFunction:" + invocationCount);
       invocationCount++;
 
       context.getResultSender().lastResult(Boolean.TRUE);
@@ -71,14 +71,15 @@ public class RestAPIsOnMembersFunctionExecutionDUnitTest extends RestAPITestBase
     }
   }
 
-  private String createCacheAndRegisterFunction(String hostName, String memberName) {
-    final int servicePort = AvailablePortHelper.getRandomAvailableTCPPort();
+  private String createCacheAndRegisterFunction(VM vm, String memberName) {
+    final String hostName = vm.getHost().getHostName();
+    final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.NAME_NAME, memberName);
     props.setProperty(DistributionConfig.START_DEV_REST_API_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_BIND_ADDRESS_NAME, hostName);
-    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(servicePort));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(serverPort));
 
     Cache c = null;
     try {
@@ -90,7 +91,7 @@ public class RestAPIsOnMembersFunctionExecutionDUnitTest extends RestAPITestBase
     c = CacheFactory.create(new RestAPIsOnMembersFunctionExecutionDUnitTest("temp").getSystem(props));
     FunctionService.registerFunction(new OnMembersFunction());
 
-    String restEndPoint = "http://" + hostName + ":" + servicePort + "/gemfire-api/v1";
+    String restEndPoint = "http://" + hostName + ":" + serverPort + "/gemfire-api/v1";
     return restEndPoint;
 
   }
@@ -114,10 +115,10 @@ public class RestAPIsOnMembersFunctionExecutionDUnitTest extends RestAPITestBase
   }
 
   private void createCacheForVMs() {
-    restURLs.add(vm0.invoke("createCacheAndRegisterFunction",() -> createCacheAndRegisterFunction(vm0.getHost().getHostName(), "m1")));
-    restURLs.add(vm1.invoke("createCacheAndRegisterFunction",() -> createCacheAndRegisterFunction(vm1.getHost().getHostName(), "m2")));
-    restURLs.add(vm2.invoke("createCacheAndRegisterFunction",() -> createCacheAndRegisterFunction(vm2.getHost().getHostName(), "m3")));
-    restURLs.add(vm3.invoke("createCacheAndRegisterFunction",() -> createCacheAndRegisterFunction(vm3.getHost().getHostName(), "m4")));
+    restURLs.add(vm0.invoke(() -> createCacheAndRegisterFunction(vm0, "m1")));
+    restURLs.add(vm1.invoke(() -> createCacheAndRegisterFunction(vm1, "m2")));
+    restURLs.add(vm2.invoke(() -> createCacheAndRegisterFunction(vm2, "m3")));
+    restURLs.add(vm3.invoke(() -> createCacheAndRegisterFunction(vm3, "m4")));
   }
 
   public void testFunctionExecutionEOnSelectedMembers() {


[32/50] [abbrv] incubator-geode git commit: GEODE-357: Remove third server from testInitiateFailoverByCacheOperationThreads_Secondary

Posted by kl...@apache.org.
GEODE-357: Remove third server from testInitiateFailoverByCacheOperationThreads_Secondary

This test was intermittently connecting to either 2 servers or 3. If the
test never connected to the third server, it would fail waiting for the
live server count to be 2.

Also cleaned up some code to use Awaitility and to have a lower ping
interval, because the ping is what causes us to disconnect from the
stopped server.


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

Branch: refs/heads/feature/GEODE-835
Commit: ef8b8b64f926039004e2b89c2eecc43bfe4dfc8d
Parents: 18761c7
Author: Dan Smith <up...@apache.org>
Authored: Tue May 17 16:52:38 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed May 18 11:29:55 2016 -0700

----------------------------------------------------------------------
 .../sockets/HAStartupAndFailoverDUnitTest.java  | 38 +++++---------------
 1 file changed, 8 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ef8b8b64/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
index 13f2527..1068534 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
@@ -19,6 +19,7 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
 import org.junit.experimental.categories.Category;
 
@@ -50,6 +51,7 @@ import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import com.jayway.awaitility.Awaitility;
 
 /**
  * Test to verify Startup. and failover during startup.
@@ -289,15 +291,16 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase {
     /**
      * Tests failover initialization by cache operation Threads on secondary
      */
-    @Category(FlakyTest.class) // GEODE-357: random ports, eats exceptions (fixed 1), time sensitive, waitForCriterions
     public void testInitiateFailoverByCacheOperationThreads_Secondary() throws Exception {
-      // create a client with large retry interval for server monitors and no client updater thread
+      //Stop the 3rd server to guarantee the client put will go to the first server
+      server3.invoke(() -> HAStartupAndFailoverDUnitTest.stopServer());
+      // create a client with no client updater thread
       // so that only cache operation can detect a server failure and should initiate failover
       createClientCacheWithLargeRetryIntervalAndWithoutCallbackConnection(this.getName()
           , NetworkUtils.getServerHostName(server1.getHost()));
       server2.invoke(() -> HAStartupAndFailoverDUnitTest.stopServer());
       put();
-      verifyDeadAndLiveServers(1,2);
+      verifyDeadAndLiveServers(1,1);
     }
 
     public static void put()
@@ -316,32 +319,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase {
     public static void verifyDeadAndLiveServers(final int expectedDeadServers, 
         final int expectedLiveServers)
   {
-    WaitCriterion wc = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        return pool.getConnectedServerCount() == expectedLiveServers;
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    Wait.waitForCriterion(wc, 60 * 1000, 1000, true);
-//     while (proxy.getDeadServers().size() != expectedDeadServers) { // wait until condition is met
-//       assertTrue("Waited over " + maxWaitTime + "for dead servers to become : " + expectedDeadServers ,
-//           //" This issue can occur on Solaris as DSM thread get stuck in connectForServer() call, and hence not recovering any newly started server. This may be beacuase of tcp_ip_abort_cinterval kernal level property on solaris which has 3 minutes as a default value ",
-//           (System.currentTimeMillis() - start) < maxWaitTime);
-//       try {
-//         Thread.yield();
-//         if(proxy.getDeadServers().size() != expectedDeadServers){
-//           synchronized(delayLock) {delayLock.wait(2000);}
-          
-//         }
-//       }
-//       catch (InterruptedException ie) {
-//         fail("Interrupted while waiting ", ie);
-//       }
-//     }
-//     start = System.currentTimeMillis();
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(expectedLiveServers, pool.getConnectedServerCount()));
   }
  
     public static void setClientServerObserver() {
@@ -641,7 +619,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase {
         .addServer(host, PORT1.intValue())
         .addServer(host, PORT2.intValue())
         .addServer(host, PORT3.intValue())
-        .setReadTimeout(10000)
+        .setPingInterval(500)
         // .setRetryInterval(200000)
         .create("HAStartupAndFailoverDUnitTestPool");
     } finally {


[22/50] [abbrv] incubator-geode git commit: GEODE-1405: Fixing shouldRecoverPersistentIndexWhenDataStillInQueue

Posted by kl...@apache.org.
GEODE-1405: Fixing shouldRecoverPersistentIndexWhenDataStillInQueue

The changes in 3e8a610e99bc005bcc56c378e3d1e2274a3d468c broke this test.
It just needs to wait for the flush *after* the data region is created.


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

Branch: refs/heads/feature/GEODE-835
Commit: 09b3c036e083080306e164a56f6a63f408b40e71
Parents: 3e181e8
Author: Dan Smith <up...@apache.org>
Authored: Tue May 17 11:43:53 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue May 17 11:43:53 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/09b3c036/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index fe754a4..ef7f808 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -185,9 +185,9 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     cache.close();
     createCache();
     createIndex("field1", "field2");
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
       .create(INDEX_NAME, REGION_NAME,
         "field1:world");


[25/50] [abbrv] incubator-geode git commit: GEODE-1209: Added new attribute to forward eviction/expiration to AEQ. Following changes are made: 1. Added new attribue/flag to forward eviction and expiration destroy events. 2. Added new option in CreateAsyn

Posted by kl...@apache.org.
GEODE-1209: Added new attribute to forward eviction/expiration to AEQ.
Following changes are made:
1. Added new attribue/flag to forward eviction and expiration destroy events.
2. Added new option in CreateAsyncEventQueue Gfsh command. Also cleaned up arguments for create AEQ function invoked by gfsh. Instead of object array the args are wrapped in args object.


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

Branch: refs/heads/feature/GEODE-835
Commit: 46056a6611970c06ab1414900e35e4135533af87
Parents: 8266f6b
Author: Anil <ag...@pivotal.io>
Authored: Tue May 3 13:52:18 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Tue May 17 15:59:39 2016 -0700

----------------------------------------------------------------------
 .../SharedConfigurationEndToEndDUnitTest.java   |   1 +
 .../cache/asyncqueue/AsyncEventQueue.java       |   9 +
 .../asyncqueue/AsyncEventQueueFactory.java      |  13 +
 .../internal/AsyncEventQueueFactoryImpl.java    |   8 +-
 .../internal/AsyncEventQueueImpl.java           |   9 +-
 .../gemfire/cache/wan/GatewaySender.java        |   2 +
 .../gemfire/internal/cache/LocalRegion.java     |  52 ++-
 .../cache/wan/AbstractGatewaySender.java        | 100 +++---
 .../cache/wan/GatewaySenderAttributes.java      |   7 +
 .../cache/xmlcache/AsyncEventQueueCreation.java |  11 +
 .../internal/cache/xmlcache/CacheXml.java       |   1 +
 .../cache/xmlcache/CacheXmlGenerator.java       |   8 +
 .../internal/cache/xmlcache/CacheXmlParser.java |   7 +
 .../internal/cli/commands/QueueCommands.java    |  18 +-
 .../functions/AsyncEventQueueFunctionArgs.java  | 134 +++++++
 .../CreateAsyncEventQueueFunction.java          |  54 ++-
 .../internal/cli/i18n/CliStrings.java           |   2 +
 .../controllers/QueueCommandsController.java    |   1 +
 .../geode.apache.org/schema/cache/cache-1.0.xsd |   1 +
 ...ventQueueEvictionAndExpirationJUnitTest.java | 346 +++++++++++++++++++
 .../cache30/CacheXmlGeode10DUnitTest.java       |  78 +++++
 .../cli/commands/QueueCommandsDUnitTest.java    |   2 +
 .../codeAnalysis/sanctionedSerializables.txt    |   1 +
 23 files changed, 747 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 a6221e9..5f3bf1f 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
@@ -226,6 +226,7 @@ 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__MAXIMUM_QUEUE_MEMORY, "1000");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, OrderPolicy.KEY.toString());
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, "true");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 a2b8b0f..c2d04a1 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,4 +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.
+   */
+  public boolean isIgnoreEvictionAndExpiration();
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 3e30b38..c607142 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
@@ -170,7 +170,20 @@ public interface AsyncEventQueueFactory {
   public AsyncEventQueueFactory setGatewayEventSubstitutionListener(
       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.
+   *
+   * @param ignore 
+   *        boolean to indicate whether to ignore eviction and expiration events. 
+   */
+  public AsyncEventQueueFactory setIgnoreEvictionAndExpiration(boolean ignore);
 
+  
   /**
    * 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/46056a66/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 312e880..1ec3ba0 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();
   }
 
   public AsyncEventQueueFactory setParallel(boolean isParallel) {
@@ -292,4 +292,10 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
     this.attrs.isMetaQueue = isMetaQueue;
     return this;
   }
+
+  @Override
+  public AsyncEventQueueFactory setIgnoreEvictionAndExpiration(boolean ignore) {
+    this.attrs.ignoreEvictionAndExpiration = ignore;
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 6b3eb4a..5a0b370 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
@@ -36,7 +36,7 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
   private GatewaySender sender = null;
   
   private AsyncEventListener asyncEventListener = null;
-  
+    
   public static final String ASYNC_EVENT_QUEUE_PREFIX = "AsyncEventQueue_";
   
   public AsyncEventQueueImpl(GatewaySender sender, AsyncEventListener eventListener) {
@@ -200,6 +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();
+   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 c5b5d3a..b0ad410 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
@@ -96,6 +96,8 @@ public interface GatewaySender {
 
   public static final int DEFAULT_DISPATCHER_THREADS = 5;
   
+  public static final boolean DEFAULT_IGNORE_EVICTION_EXPIRATION = true;
+  
   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/46056a66/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 e9f5819..b5ff7ee 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,8 +1147,7 @@ public class LocalRegion extends AbstractRegion
   @Override
   public boolean generateEventID()
   {     
-    return !(isUsedForPartitionedRegionAdmin()
-        || isUsedForPartitionedRegionBucket() );
+    return !isUsedForPartitionedRegionAdmin();
   }
 
   public final Object destroy(Object key, Object aCallbackArgument)
@@ -6647,10 +6646,10 @@ public class LocalRegion extends AbstractRegion
   protected void notifyGatewaySender(EnumListenerEvent operation,
       EntryEventImpl event) {
     
-    if (event.isConcurrencyConflict()) { // usually concurrent cache modification problem
+    if (isPdxTypesRegion() || event.isConcurrencyConflict() /* usually concurrent cache modification problem */) { 
       return;
     }
-    
+
     // Return if the inhibit all notifications flag is set
     if (event.inhibitAllNotifications()){
       if(logger.isDebugEnabled()) {
@@ -6659,34 +6658,31 @@ public class LocalRegion extends AbstractRegion
       return;
     }
     
-    if (!event.getOperation().isLocal()) {
-      Set<String> allGatewaySenderIds = null;
-      checkSameSenderIdsAvailableOnAllNodes();
-      if (event.getOperation() == Operation.UPDATE_VERSION_STAMP) {
-        allGatewaySenderIds = getGatewaySenderIds();
-      } else {
-        allGatewaySenderIds = getAllGatewaySenderIds();
-      }
-      
-      List<Integer> allRemoteDSIds = getRemoteDsIds(allGatewaySenderIds);
+    
+    Set<String> allGatewaySenderIds = null;
+    checkSameSenderIdsAvailableOnAllNodes();
+    if (event.getOperation() == Operation.UPDATE_VERSION_STAMP) {
+      allGatewaySenderIds = getGatewaySenderIds();
+    } else {
+      allGatewaySenderIds = getAllGatewaySenderIds();
+    }
 
-      if (allRemoteDSIds != null) {
-        for (GatewaySender sender : getCache().getAllGatewaySenders()) {
-          if (!isPdxTypesRegion()) {
-            if (allGatewaySenderIds.contains(sender.getId())) {
-              //TODO: This is a BUG. Why return and not continue?
-              if((!this.getDataPolicy().withStorage()) && sender.isParallel()){
-                return;
-              }
-              if(logger.isDebugEnabled()) {
-                logger.debug("Notifying the GatewaySender : {}", sender.getId());
-              }
-              ((AbstractGatewaySender)sender).distribute(operation, event,
-                  allRemoteDSIds);
-            }
+    List<Integer> allRemoteDSIds = getRemoteDsIds(allGatewaySenderIds);
+    if (allRemoteDSIds != null) {
+      for (GatewaySender sender : getCache().getAllGatewaySenders()) {
+        if (allGatewaySenderIds.contains(sender.getId())) {
+          //TODO: This is a BUG. Why return and not continue?
+          if((!this.getDataPolicy().withStorage()) && sender.isParallel()){
+            return;
+          }
+          if(logger.isDebugEnabled()) {
+            logger.debug("Notifying the GatewaySender : {}", sender.getId());
           }
+          ((AbstractGatewaySender)sender).distribute(operation, event,
+              allRemoteDSIds);
         }
       }
+
       
 //      if (shouldNotifyGatewaySender()) {
 //        // Get All WAN site DSID's to be sent to each WAN site so that they

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 fe09d03..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
@@ -34,6 +34,7 @@ import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
@@ -137,6 +138,8 @@ public abstract class AbstractGatewaySender implements GatewaySender,
 
   protected List<AsyncEventListener> listeners;
   
+  protected boolean ignoreEvictionAndExpiration;
+  
   protected GatewayEventSubstitutionFilter substitutionFilter;
   
   protected LocatorDiscoveryCallback locatorDiscoveryCallback;
@@ -269,55 +272,7 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       initializeEventIdIndex();
     }
     this.isBucketSorted = attrs.isBucketSorted();
-  }
-  
-  public void createSender(Cache cache, GatewaySenderAttributes attrs){
-    this.cache = cache;
-    this.id = attrs.getId();
-    this.socketBufferSize = attrs.getSocketBufferSize();
-    this.socketReadTimeout = attrs.getSocketReadTimeout();
-    this.queueMemory = attrs.getMaximumQueueMemory();
-    this.batchSize = attrs.getBatchSize();
-    this.batchTimeInterval = attrs.getBatchTimeInterval();
-    this.isConflation = attrs.isBatchConflationEnabled();
-    this.isPersistence = attrs.isPersistenceEnabled();
-    this.alertThreshold = attrs.getAlertThreshold();
-    this.manualStart = attrs.isManualStart();
-    this.isParallel = attrs.isParallel();
-    this.isForInternalUse = attrs.isForInternalUse();
-    this.diskStoreName = attrs.getDiskStoreName();
-    this.remoteDSId = attrs.getRemoteDSId();
-    this.eventFilters = attrs.getGatewayEventFilters();
-    this.transFilters = attrs.getGatewayTransportFilters();
-    this.listeners = attrs.getAsyncEventListeners();
-    this.substitutionFilter = attrs.getGatewayEventSubstitutionFilter();
-    this.locatorDiscoveryCallback = attrs.getGatewayLocatoDiscoveryCallback();
-    this.isDiskSynchronous = attrs.isDiskSynchronous();
-    this.policy = attrs.getOrderPolicy();
-    this.dispatcherThreads = attrs.getDispatcherThreads();
-    this.parallelismForReplicatedRegion = attrs.getParallelismForReplicatedRegion();
-    //divide the maximumQueueMemory of sender equally using number of dispatcher threads.
-    //if dispatcherThreads is 1 then maxMemoryPerDispatcherQueue will be same as maximumQueueMemory of sender
-    this.maxMemoryPerDispatcherQueue = this.queueMemory / this.dispatcherThreads;
-    this.myDSId = InternalDistributedSystem.getAnyInstance().getDistributionManager().getDistributedSystemId();
-    this.serialNumber = DistributionAdvisor.createSerialNumber();
-    if (!(this.cache instanceof CacheCreation)) {
-      this.stopper = new Stopper(cache.getCancelCriterion());
-      this.senderAdvisor = GatewaySenderAdvisor.createGatewaySenderAdvisor(this);
-      if (!this.isForInternalUse()) {
-        this.statistics = new AsyncEventQueueStats(cache.getDistributedSystem(),
-            id);
-      }
-      else {// this sender lies underneath the AsyncEventQueue. Need to have
-            // AsyncEventQueueStats
-        this.statistics = new AsyncEventQueueStats(
-            cache.getDistributedSystem(), AsyncEventQueueImpl
-                .getAsyncEventQueueIdFromSenderId(id));
-      }
-      initializeEventIdIndex();
-    }
-    this.isBucketSorted = attrs.isBucketSorted();
-
+    this.ignoreEvictionAndExpiration = attrs.isIgnoreEvictionAndExpiration();
   }
   
   public GatewaySenderAdvisor getSenderAdvisor() {
@@ -392,6 +347,10 @@ public abstract class AbstractGatewaySender implements GatewaySender,
     return !this.listeners.isEmpty();
   }
   
+  public boolean isIgnoreEvictionAndExpiration() {
+    return this.ignoreEvictionAndExpiration;
+  }
+  
   public boolean isManualStart() {
     return this.manualStart;
   }
@@ -839,16 +798,49 @@ public abstract class AbstractGatewaySender implements GatewaySender,
     return this.eventProcessor;
   }
 
+  /**
+   * Check if this event can be distributed by senders.
+   * @param event
+   * @param stats
+   * @return boolean True if the event is allowed.
+   */
+  private boolean checkForDistribution(EntryEventImpl event, GatewaySenderStats stats) {
+    if (event.getRegion().getDataPolicy().equals(DataPolicy.NORMAL))
+    {
+      return false;
+    }
+    
+    // Eviction and expirations are not passed to WAN.
+    // Eviction and Expiration are passed to AEQ based on its configuration.
+    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()) {
+        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) {
+        logger.debug("Returning back without putting into the gateway sender queue");
+      }
+      return;
+    }
+    
     final GatewaySenderStats stats = getStatistics();
     stats.incEventsReceived();
-    // If the event is local (see bug 35831) or an expiration ignore it.
-    //removed the check of isLocal as in notifyGAtewayHub this has been taken care
-    if (/*event.getOperation().isLocal() || */event.getOperation().isExpiration()
-        || event.getRegion().getDataPolicy().equals(DataPolicy.NORMAL)) {
+   
+    if (!checkForDistribution(event, stats)) {
       getStatistics().incEventsNotQueued();
       return;
     }
@@ -941,6 +933,7 @@ public abstract class AbstractGatewaySender implements GatewaySender,
     }
     try {
       // If this gateway is not running, return
+      // The sender may have stopped, after we have checked the status in the beginning. 
       if (!isRunning()) {
         if (isDebugEnabled) {
           logger.debug("Returning back without putting into the gateway sender queue");
@@ -988,6 +981,7 @@ public abstract class AbstractGatewaySender implements GatewaySender,
     }
   }
   
+
   /**
    * During sender is getting started, if there are any cache operation on queue then that event will be stored in temp queue. 
    * Once sender is started, these event from tmp queue will be added to sender queue.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 1cef940..163943f 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
@@ -83,6 +83,8 @@ public class GatewaySenderAttributes {
   
   public boolean isMetaQueue = GatewaySenderAttributes.DEFAULT_IS_META_QUEUE;
   
+  public boolean ignoreEvictionAndExpiration = GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION;
+  
   public int getSocketBufferSize() {
     return this.socketBufferSize;
   }
@@ -192,4 +194,9 @@ public class GatewaySenderAttributes {
   public boolean isMetaQueue() {
     return this.isMetaQueue;
   }
+  
+  public boolean isIgnoreEvictionAndExpiration() {
+    return this.ignoreEvictionAndExpiration;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 0015665..4c2943e 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,6 +43,7 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
   private boolean isBucketSorted = false;
   private int dispatcherThreads = 1;
   private OrderPolicy orderPolicy = OrderPolicy.KEY;
+  private boolean ignoreEvictionAndExpiration = true;
   
   public AsyncEventQueueCreation() {
   }
@@ -62,6 +63,7 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
     this.asyncEventListener = eventListener;
     this.isBucketSorted = senderAttrs.isBucketSorted; 
     this.gatewayEventSubstitutionFilter = senderAttrs.eventSubstitutionFilter;
+    this.ignoreEvictionAndExpiration = senderAttrs.ignoreEvictionAndExpiration;
   }
   
   @Override
@@ -211,4 +213,13 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
   public void setBucketSorted(boolean isBucketSorted) {
     this.isBucketSorted = isBucketSorted;
   }
+
+  public void setIgnoreEvictionAndExpiration(boolean ignore) {
+    this.ignoreEvictionAndExpiration = ignore;
+  }
+  
+  @Override
+  public boolean isIgnoreEvictionAndExpiration() {
+    return this.ignoreEvictionAndExpiration;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 aa7d49a..c3eccd2 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
@@ -762,6 +762,7 @@ 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";
   
   /** The name of the <code>compressor</code> attribute */
   protected static final String COMPRESSOR = "compressor";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 ea3c975..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
@@ -1521,10 +1521,18 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
         atts.addAttribute("", "", ORDER_POLICY, "", String.valueOf(asyncEventQueue
           .getOrderPolicy()));
       }
+      // 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()));
+      }
       // 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/46056a66/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 f344938..aec2dc3 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
@@ -2313,6 +2313,12 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
       }
     }
     
+    // forward eviction and expiration events.
+    String ignoreEvictionExpiration = atts.getValue(IGNORE_EVICTION_AND_EXPIRATION);
+    if (ignoreEvictionExpiration != null) {
+      asyncEventQueueCreation.setIgnoreEvictionAndExpiration(Boolean.parseBoolean(ignoreEvictionExpiration));
+    }
+    
     stack.push(asyncEventQueueCreation);
   }
   
@@ -2346,6 +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());
     List<GatewayEventFilter> gatewayEventFilters = asyncEventChannelCreation.getGatewayEventFilters();
     for (GatewayEventFilter gatewayEventFilter : gatewayEventFilters) {
       factory.addGatewayEventFilter(gatewayEventFilter);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 89534a6..b9c853d 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
@@ -25,6 +25,7 @@ import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.cli.Result.Status;
 import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.domain.AsyncEventQueueDetails;
+import com.gemstone.gemfire.management.internal.cli.functions.AsyncEventQueueFunctionArgs;
 import com.gemstone.gemfire.management.internal.cli.functions.CliFunctionResult;
 import com.gemstone.gemfire.management.internal.cli.functions.CreateAsyncEventQueueFunction;
 import com.gemstone.gemfire.management.internal.cli.functions.ListAsyncEventQueuesFunction;
@@ -36,6 +37,7 @@ import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationWriter;
 import com.gemstone.gemfire.management.internal.configuration.domain.XmlEntity;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
@@ -105,6 +107,11 @@ public class QueueCommands implements CommandMarker {
                  specifiedDefaultValue = "true",
                  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) 
+      Boolean ignoreEvictionAndExpiration,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY,
                  unspecifiedDefaultValue = "100", 
                  help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP)
@@ -159,8 +166,15 @@ public class QueueCommands implements CommandMarker {
         return crex.getResult();
       }
 
-      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateAsyncEventQueueFunction(), new Object[] { id, parallel, enableBatchConflation, batchSize,batchTimeInterval,
-          persistent, diskStore, diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy, gatewayEventFilters, gatewaySubstitutionListener, listener, listenerProperties }, targetMembers);
+      AsyncEventQueueFunctionArgs aeqArgs = new AsyncEventQueueFunctionArgs(id, parallel, 
+          enableBatchConflation, batchSize,batchTimeInterval,
+          persistent, diskStore, diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy, 
+          gatewayEventFilters, gatewaySubstitutionListener, listener, listenerProperties, 
+          ignoreEvictionAndExpiration);
+
+      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateAsyncEventQueueFunction(), 
+          aeqArgs, targetMembers);
+
       List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());
 
       XmlEntity xmlEntity = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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
new file mode 100644
index 0000000..2066628
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
@@ -0,0 +1,134 @@
+/*
+ * 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.management.internal.cli.functions;
+
+import java.io.Serializable;
+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;
+  private boolean isParallel;
+  private boolean enableBatchConflation;
+  private int batchSize;
+  private int batchTimeInterval;
+  private boolean persistent;
+  private String diskStoreName;
+  private boolean diskSynchronous;
+  private int maxQueueMemory;
+  private int dispatcherThreads; 
+  private String orderPolicy;
+  private String[] gatewayEventFilters;
+  private String gatewaySubstitutionFilter;
+  private String listenerClassName;
+  private Properties listenerProperties;
+  private boolean ignoreEvictionAndExpiration;
+
+  public AsyncEventQueueFunctionArgs(String asyncEventQueueId,
+      boolean isParallel, boolean enableBatchConflation, int batchSize,
+      int batchTimeInterval, boolean persistent, String diskStoreName,
+      boolean diskSynchronous, int maxQueueMemory, int dispatcherThreads,
+      String orderPolicy, String[] gatewayEventFilters,
+      String gatewaySubstitutionFilter, String listenerClassName,
+      Properties listenerProperties, boolean ignoreEvictionAndExpiration) {
+    this.asyncEventQueueId = asyncEventQueueId;
+    this.isParallel = isParallel;
+    this.enableBatchConflation = enableBatchConflation;
+    this.batchSize = batchSize;
+    this.batchTimeInterval = batchTimeInterval;
+    this.persistent = persistent;
+    this.diskStoreName = diskStoreName;
+    this.diskSynchronous = diskSynchronous;
+    this.maxQueueMemory = maxQueueMemory;
+    this.dispatcherThreads = dispatcherThreads;
+    this.orderPolicy = orderPolicy;
+    this.gatewayEventFilters = gatewayEventFilters;
+    this.gatewaySubstitutionFilter = gatewaySubstitutionFilter;
+    this.listenerClassName = listenerClassName;
+    this.listenerProperties = listenerProperties;
+    this.ignoreEvictionAndExpiration = ignoreEvictionAndExpiration;
+  }
+
+  public String getAsyncEventQueueId() {
+    return asyncEventQueueId;
+  }
+
+  public boolean isParallel() {
+    return isParallel;
+  }
+
+  public boolean isEnableBatchConflation() {
+    return enableBatchConflation;
+  }
+
+  public int getBatchSize() {
+    return batchSize;
+  }
+
+  public int getBatchTimeInterval() {
+    return batchTimeInterval;
+  }
+
+  public boolean isPersistent() {
+    return persistent;
+  }
+
+  public String getDiskStoreName() {
+    return diskStoreName;
+  }
+
+  public boolean isDiskSynchronous() {
+    return diskSynchronous;
+  }
+
+  public int getMaxQueueMemory() {
+    return maxQueueMemory;
+  }
+
+  public int getDispatcherThreads() {
+    return dispatcherThreads;
+  }
+
+  public String getOrderPolicy() {
+    return orderPolicy;
+  }
+
+  public String[] getGatewayEventFilters() {
+    return gatewayEventFilters;
+  }
+
+  public String getGatewaySubstitutionFilter() {
+    return gatewaySubstitutionFilter;
+  }
+
+  public String getListenerClassName() {
+    return listenerClassName;
+  }
+
+  public Properties getListenerProperties() {
+    return listenerProperties;
+  }
+
+  public boolean isIgnoreEvictionAndExpiration() {
+    return ignoreEvictionAndExpiration;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 7481c0e..695fbfb 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
@@ -62,23 +62,8 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
     String memberId = "";
 
     try {
-      final Object[] args = (Object[]) context.getArguments();
-      final String asyncEventQueueId = (String) args[0];
-      final boolean isParallel = (Boolean) args[1];
-      final boolean enableBatchConflation = (Boolean) args[2];
-      final int batchSize = (Integer) args[3];
-      final int batchTimeInterval =(Integer) args[4];
-      final boolean persistent = (Boolean) args[5];
-      final String diskStoreName = (String) args[6];
-      final boolean diskSynchronous =(Boolean) args[7];
-      final int maxQueueMemory = (Integer) args[8];
-      final int dispatcherThreads =(Integer) args[9]; 
-      final String orderPolicy= (String) args[10];
-      final String[] gatewayEventFilters =(String[]) args[11];
-      final String gatewaySubstitutionFilter = (String) args[12];
-      final String listenerClassName = (String) args[13];
-      final Properties listenerProperties = (Properties) args[14];
-
+      AsyncEventQueueFunctionArgs aeqArgs =  (AsyncEventQueueFunctionArgs)context.getArguments();
+      
       GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
 
       DistributedMember member = cache.getDistributedSystem().getDistributedMember();
@@ -89,32 +74,39 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
         memberId = member.getName();
       }
 
-      AsyncEventQueueFactory asyncEventQueueFactory = cache.createAsyncEventQueueFactory();
-      asyncEventQueueFactory.setParallel(isParallel);
-      asyncEventQueueFactory.setBatchConflationEnabled(enableBatchConflation);
-      asyncEventQueueFactory.setBatchSize(batchSize);
-      asyncEventQueueFactory.setBatchTimeInterval(batchTimeInterval);
-      asyncEventQueueFactory.setPersistent(persistent);
-      asyncEventQueueFactory.setDiskStoreName(diskStoreName);
-      asyncEventQueueFactory.setDiskSynchronous(diskSynchronous);
-      asyncEventQueueFactory.setMaximumQueueMemory(maxQueueMemory);
-      asyncEventQueueFactory.setDispatcherThreads(dispatcherThreads);
-      asyncEventQueueFactory.setOrderPolicy(OrderPolicy.valueOf(orderPolicy));
+      AsyncEventQueueFactory asyncEventQueueFactory = cache.createAsyncEventQueueFactory()
+          .setParallel(aeqArgs.isParallel())
+          .setBatchConflationEnabled(aeqArgs.isEnableBatchConflation())
+          .setBatchSize(aeqArgs.getBatchSize())
+          .setBatchTimeInterval(aeqArgs.getBatchTimeInterval())
+          .setPersistent(aeqArgs.isPersistent())
+          .setDiskStoreName(aeqArgs.getDiskStoreName())
+          .setDiskSynchronous(aeqArgs.isDiskSynchronous())
+          .setIgnoreEvictionAndExpiration(aeqArgs.isIgnoreEvictionAndExpiration())
+          .setMaximumQueueMemory(aeqArgs.getMaxQueueMemory())
+          .setDispatcherThreads(aeqArgs.getDispatcherThreads())
+          .setOrderPolicy(OrderPolicy.valueOf(aeqArgs.getOrderPolicy()));
+
+      String[] gatewayEventFilters = aeqArgs.getGatewayEventFilters();
       if (gatewayEventFilters != null) {
         for (String gatewayEventFilter : gatewayEventFilters) {
           Class<?> gatewayEventFilterKlass = forName(gatewayEventFilter, CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER);
           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);
       listenerInstance = listenerClass.newInstance();
 
+      Properties listenerProperties = aeqArgs.getListenerProperties();
       if (listenerProperties != null && !listenerProperties.isEmpty()) {
         if (!(listenerInstance instanceof Declarable)) {
           throw new IllegalArgumentException("Listener properties were provided, but the listener specified does not implement Declarable.");
@@ -127,9 +119,9 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
         cache.addDeclarableProperties(declarablesMap);
       }
 
-      asyncEventQueueFactory.create(asyncEventQueueId, (AsyncEventListener) listenerInstance);
+      asyncEventQueueFactory.create(aeqArgs.getAsyncEventQueueId(), (AsyncEventListener) listenerInstance);
 
-      XmlEntity xmlEntity = new XmlEntity(CacheXml.ASYNC_EVENT_QUEUE, "id", asyncEventQueueId);
+      XmlEntity xmlEntity = new XmlEntity(CacheXml.ASYNC_EVENT_QUEUE, "id", aeqArgs.getAsyncEventQueueId());
       context.getResultSender().lastResult(new CliFunctionResult(memberId, xmlEntity, "Success"));
 
     } catch (CacheClosedException cce) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 28ba856..241c9e2 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
@@ -423,6 +423,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__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/46056a66/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 b8353fa..83d7bc1 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,6 +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__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,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 cc6d189..688ff1f 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,6 +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:complexType>
         </xsd:element>
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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
new file mode 100644
index 0000000..5efac05
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
@@ -0,0 +1,346 @@
+/*
+ * 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.cache.asyncqueue;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+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.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.jayway.awaitility.Awaitility;
+
+import static org.mockito.Mockito.*;
+
+@Category(IntegrationTest.class)
+public class AsyncEventQueueEvictionAndExpirationJUnitTest {
+  
+  private AsyncEventQueue aeq;
+  private Cache cache;
+  
+  @Rule 
+  public TestName name = new TestName();
+  
+  @Before
+  public void getCache() {
+    try {
+       cache = CacheFactory.getAnyInstance();
+    } catch (Exception e) {
+      //ignore
+    }
+    if (null == cache) {
+      cache = (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").create();
+    }
+  }
+
+  @After
+  public void destroyCache() {
+    if (cache != null && !cache.isClosed()) {
+      cache.close();
+      cache = null;
+    }
+  }
+
+  
+  @Test
+  public void isIgnoreEvictionAndExpirationAttributeTrueByDefault() {
+    AsyncEventListener al = mock(AsyncEventListener.class);
+    aeq = cache.createAsyncEventQueueFactory().create("aeq", al);
+    // Test for default value of isIgnoreEvictionAndExpiration setting.
+    assertTrue(aeq.isIgnoreEvictionAndExpiration());
+  }
+  
+  @Test
+  public void canSetFalseForIgnoreEvictionAndExpiration() {
+    AsyncEventListener al = mock(AsyncEventListener.class);
+    aeq = cache.createAsyncEventQueueFactory().setIgnoreEvictionAndExpiration(false).create("aeq", al);
+    // Test for default value of isIgnoreEvictionAndExpiration setting.
+    assertFalse(aeq.isIgnoreEvictionAndExpiration());
+  }
+  
+  
+  @Test
+  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 */);
+  }
+
+  @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 */);
+  }
+
+  @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 */);
+  }
+
+  @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 */);
+  }
+
+  @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 */);
+  }
+
+  @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 */);
+  }
+  
+  @Test
+  public void evictionPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+    // 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 */);
+  }
+
+  @Test
+  public void evictionPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+    // 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 */);
+  }
+
+  @Test
+  public void overflowNotPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+    // 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 */);
+  }
+
+  @Test
+  public void overflowNotPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+    // 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 */);
+  }
+
+  @Test
+  public void expirationDestroyPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+    // 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 */);    
+  }
+
+  @Test
+  public void expirationDestroyPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+    // 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 */);    
+  }
+
+  @Test
+  public void expirationInvalidateNotPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+    // 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 */);
+  }
+
+  @Test
+  public void expirationInvalidateNotPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+    // 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 */);
+  }
+
+  
+  
+  private void createPopulateAndVerifyEvents(boolean isPR, boolean ignoreEvictionExpiration, int expectedEvents, boolean eviction, boolean evictionOverflow, 
+      boolean expirationDestroy, boolean expirationInvalidate, boolean checkForDestroyOp, boolean checkForInvalidateOp) {
+    
+    // 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");
+    
+    // The AQListner should get two events. One for create, one for destroy.
+    Awaitility.await().atMost(100, TimeUnit.SECONDS).until(() -> {return (events.size() == expectedEvents);});
+    
+    // Check for the expected operation.
+    if (checkForDestroyOp) {
+      assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
+    }
+
+    if (checkForInvalidateOp) {
+      assertTrue("Invalidate event not arrived", checkForOperation(events, true, false));
+    }
+    
+    // Test complete. Destroy region.
+    r.destroyRegion();
+  }
+
+  private boolean checkForOperation(List<AsyncEvent> events, boolean invalidate, boolean destroy) {
+    boolean found = false;
+    for (AsyncEvent e : events) {
+      if (invalidate && e.getOperation().isInvalidate()) {
+        found = true;
+        break;
+      }
+      if (destroy && e.getOperation().isDestroy()) {
+        found = true;
+        break;
+      }
+    }
+    return found;
+  }
+
+  private void createAsyncEventQueue(String id, boolean ignoreEvictionExpiration, List<AsyncEvent> storeEvents) {
+    AsyncEventListener al = this.createAsyncListener(storeEvents);
+    aeq = cache.createAsyncEventQueueFactory().setParallel(false)
+        .setIgnoreEvictionAndExpiration(ignoreEvictionExpiration)
+        .setBatchSize(1).setBatchTimeInterval(1).create(id, al);
+  }
+  
+  private Region createRegion(String name, boolean isPR, String aeqId, boolean evictionDestroy, 
+      boolean evictionOverflow, boolean expirationDestroy, boolean expirationInvalidate) {
+    RegionFactory rf;
+    if (isPR) {
+      rf = cache.createRegionFactory(RegionShortcut.PARTITION);
+    } else {
+      rf = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    }
+    // Set AsyncQueue.
+    rf.addAsyncEventQueueId(aeqId);
+    if (evictionDestroy) {
+      rf.setEvictionAttributes(EvictionAttributes.createLIFOEntryAttributes(1, EvictionAction.LOCAL_DESTROY));
+    }
+    if (evictionOverflow) {
+      rf.setEvictionAttributes(EvictionAttributes.createLIFOEntryAttributes(1, EvictionAction.OVERFLOW_TO_DISK)); 
+    }
+    if (expirationDestroy) {
+      rf.setEntryTimeToLive(new ExpirationAttributes(1, ExpirationAction.DESTROY));
+    }
+    if (expirationInvalidate) {
+      rf.setEntryTimeToLive(new ExpirationAttributes(1, ExpirationAction.INVALIDATE));
+    }
+    
+    return rf.create(name);
+  }
+  
+  private AsyncEventListener createAsyncListener(List<AsyncEvent> list) {
+    AsyncEventListener listener = new AsyncEventListener() {
+      private List<AsyncEvent> aeList = list;
+      
+      @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);
+        return true;
+      }
+    };
+    return listener;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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) {
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 db14fdc..4472193 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
@@ -165,6 +165,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__LISTENER,
         "com.qcdunit.QueueCommandsDUnitTestHelper");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE, "param1");
@@ -206,6 +207,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
         assertEquals(queue.getGatewayEventSubstitutionFilter().getClass().getName(),
             "com.qcdunit.QueueCommandsDUnitTestHelper");
         assertEquals(queue.isDiskSynchronous(), false);
+        assertEquals(queue.isIgnoreEvictionAndExpiration(), false);
         assertEquals(queue.getAsyncEventListener().getClass().getName(), "com.qcdunit.QueueCommandsDUnitTestHelper");
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46056a66/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 1ee16d5..b8ef985 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
@@ -678,6 +678,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/ChangeLogLevelFunction,true,1
 com/gemstone/gemfire/management/internal/cli/functions/CloseDurableClientFunction,true,1
 com/gemstone/gemfire/management/internal/cli/functions/CloseDurableCqFunction,true,1


[10/50] [abbrv] incubator-geode git commit: GEODE-1390: Disabling broken test

Posted by kl...@apache.org.
GEODE-1390: Disabling broken test


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

Branch: refs/heads/feature/GEODE-835
Commit: 08ef0c1ee442d33e2da872a4a5e4bdb120d25817
Parents: 884cf13
Author: Jens Deppe <jd...@pivotal.io>
Authored: Sat May 14 15:14:12 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Sat May 14 15:14:12 2016 -0700

----------------------------------------------------------------------
 .../cli/commands/HelpCommandsJUnitTest.java       | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/08ef0c1e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
index 7cbe85a..1d81d2e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
@@ -25,7 +25,9 @@ import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -41,8 +43,10 @@ public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
 
   private int jmxPort;
 
+  private Gfsh gfsh;
+
   @Before
-  public void setup() {
+  public void setup() throws Exception {
     jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
     Properties localProps = new Properties();
@@ -51,15 +55,22 @@ public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
     localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
     getSystem(localProps);
 
+    gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
   }
 
+  @After
+  public void teardown() {
+    disconnectAllFromDS();
+
+    gfsh.executeCommand("disconnect");
+  }
+
+  @Ignore("Disconnect command doesn't appear to be working")
   @Test
   public void testOfflineHelp() throws Exception {
     Properties helpProps = new Properties();
     helpProps.load(HelpCommandsJUnitTest.class.getResourceAsStream("golden-help-offline.properties"));
 
-    Gfsh gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
-
     CommandManager cm = CommandManager.getInstance();
     for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
       // Mock commands may have been produced in the VM by other tests
@@ -88,7 +99,6 @@ public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
     Properties helpProps = new Properties();
     helpProps.load(HelpCommandsJUnitTest.class.getResourceAsStream("golden-help-online.properties"));
 
-    Gfsh gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
     gfsh.executeCommand("connect --jmx-manager=localhost[" + jmxPort + "]");
 
     CommandManager cm = CommandManager.getInstance();


[06/50] [abbrv] incubator-geode git commit: GEODE-11: Adding tests of fixed partitions and persistent lucene indexes

Posted by kl...@apache.org.
GEODE-11: Adding tests of fixed partitions and persistent lucene indexes

In addition to the tests, I fixed some code to properly propagate the
disk synchronous flag to the index.


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

Branch: refs/heads/feature/GEODE-835
Commit: 7908467a38932edcd781c6c276791e6371522269
Parents: d216e55
Author: Dan Smith <up...@apache.org>
Authored: Thu May 12 15:36:38 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri May 13 11:37:02 2016 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/test/fake/Fakes.java   |  19 +++
 .../gemfire/test/junit/rules/DiskDirRule.java   |  54 ++++++++
 .../test/junit/rules/DiskDirRuleTest.java       |  49 +++++++
 .../LuceneIndexForPartitionedRegion.java        |  28 ++--
 .../lucene/internal/LuceneQueryFactoryImpl.java |   3 +
 .../internal/PartitionedRepositoryManager.java  |   1 +
 .../LuceneIndexCreationIntegrationTest.java     | 131 +++++++++++++++++--
 .../cache/lucene/LuceneIntegrationTest.java     |  14 +-
 .../LuceneIndexForPartitionedRegionTest.java    |  58 ++++----
 .../LuceneQueryFactoryImplJUnitTest.java        |  21 ++-
 10 files changed, 312 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
index 99644b7..2a1fd8e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/fake/Fakes.java
@@ -23,6 +23,10 @@ import java.net.UnknownHostException;
 import org.junit.Assert;
 
 import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.distributed.internal.DSClock;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -96,6 +100,21 @@ public class Fakes {
     return cache().getDistributedSystem();
   }
 
+  /**
+   * A fake region, which contains a fake cache and some other
+   * fake attributes
+   */
+  public static Region region(String name, Cache cache) {
+    Region region = mock(Region.class);
+    RegionAttributes attributes = mock(RegionAttributes.class);
+    DataPolicy policy = mock(DataPolicy.class);
+    when(region.getAttributes()).thenReturn(attributes);
+    when(attributes.getDataPolicy()).thenReturn(policy);
+    when(region.getCache()).thenReturn(cache);
+    when(region.getRegionService()).thenReturn(cache);
+    return region;
+  }
+
   private Fakes() {
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
new file mode 100644
index 0000000..184619f
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
@@ -0,0 +1,54 @@
+/*
+ * 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.test.junit.rules;
+
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+
+import org.junit.rules.ExternalResource;
+
+public class DiskDirRule extends ExternalResource {
+  private File diskDir;
+
+  @Override protected void before() throws Throwable {
+    diskDir = new File(".", "DiskDirRule-" + System.nanoTime());
+  }
+
+  @Override protected void after() {
+    try {
+      Files.walk(diskDir.toPath()).forEach((path) -> {
+        try {
+          Files.delete(path);
+        }
+        catch (IOException e) {
+          //Ignore
+        }
+      });
+    } catch(IOException e) {
+      throw new RuntimeException("Could not delete disk dir: " + diskDir, e);
+    }
+    diskDir.delete();
+  }
+
+  public File get() {
+    diskDir.mkdirs();
+    return diskDir;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java
new file mode 100644
index 0000000..8c322a4
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/rules/DiskDirRuleTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.test.junit.rules;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class DiskDirRuleTest {
+  @Test
+  public void shouldDeleteDirInAfter() throws Throwable {
+    DiskDirRule diskDirRule = new DiskDirRule();
+    diskDirRule.before();
+    final File dir = diskDirRule.get();
+    assertTrue(dir.exists());
+    final File file1 = new File(dir, "file1");
+    final File subdir = new File(dir, "subdir");
+    final File file2 = new File(dir, "file2");
+    subdir.mkdir();
+    Files.write(file1.toPath(), Arrays.asList("abc"));
+    Files.write(file2.toPath(), Arrays.asList("stuff"));
+    diskDirRule.after();
+    assertFalse(dir.exists());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index dd9d384..d22ca4a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -29,7 +29,6 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
@@ -88,46 +87,37 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       repositoryManager = new PartitionedRepositoryManager(dataRegion, (PartitionedRegion)fileRegion, (PartitionedRegion)chunkRegion, mapper, analyzer);
       
       // create AEQ, AEQ listener and specify the listener to repositoryManager
-      if (withPersistence) {
-        createAEQWithPersistence();
-      }
-      else {
-        createAEQ();
-      }
+      createAEQ(dataRegion);
 
       addExtension(dataRegion);
       hasInitialized = true;
     }
   }
 
-  private AsyncEventQueueFactoryImpl createAEQFactory() {
+  private AsyncEventQueueFactoryImpl createAEQFactory(final Region dataRegion) {
     AsyncEventQueueFactoryImpl factory = (AsyncEventQueueFactoryImpl) cache.createAsyncEventQueueFactory();
     factory.setParallel(true); // parallel AEQ for PR
     factory.setMaximumQueueMemory(1000);
     factory.setDispatcherThreads(1);
     factory.setIsMetaQueue(true);
+    if(dataRegion.getAttributes().getDataPolicy().withPersistence()) {
+      factory.setPersistent(true);
+    }
+    factory.setDiskSynchronous(dataRegion.getAttributes().isDiskSynchronous());
     return factory;
   }
 
-  AsyncEventQueue createAEQWithPersistence() {
-    AsyncEventQueueFactoryImpl factory = createAEQFactory();
-    factory.setPersistent(true);
-    return createAEQ(factory);
-  }
-
-  AsyncEventQueue createAEQ() {
-    return createAEQ(createAEQFactory());
+  AsyncEventQueue createAEQ(Region dataRegion) {
+    return createAEQ(createAEQFactory(dataRegion));
   }
 
   private AsyncEventQueue createAEQ(AsyncEventQueueFactoryImpl factory) {
     LuceneEventListener listener = new LuceneEventListener(repositoryManager);
     String aeqId = LuceneServiceImpl.getUniqueIndexName(getName(), regionPath);
-    AsyncEventQueueImpl aeq = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
     AsyncEventQueue indexQueue = factory.create(aeqId, listener);
     return indexQueue;
   }
 
-
   boolean fileRegionExists(String fileRegionName) {
     return cache.<String, File> getRegion(fileRegionName) != null;
   }
@@ -156,7 +146,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     return LuceneServiceImpl.getUniqueIndexName(indexName, regionPath) + ".chunks";
   }
 
-  private PartitionAttributesFactory configureLuceneRegionAttributesFactory(PartitionAttributesFactory attributesFactory, PartitionAttributes dataRegionAttributes) {
+  private PartitionAttributesFactory configureLuceneRegionAttributesFactory(PartitionAttributesFactory attributesFactory, PartitionAttributes<?,?> dataRegionAttributes) {
     attributesFactory.setTotalNumBuckets(dataRegionAttributes.getTotalNumBuckets());
     attributesFactory.setRedundantCopies(dataRegionAttributes.getRedundantCopies());
     return attributesFactory;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
index c6087ea..385b226 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImpl.java
@@ -54,6 +54,9 @@ public class LuceneQueryFactoryImpl implements LuceneQueryFactory {
   
   public <K, V> LuceneQuery<K, V> create(String indexName, String regionName, LuceneQueryProvider provider) {
     Region<K, V> region = cache.getRegion(regionName);
+    if(region == null) {
+      throw new IllegalArgumentException("Region not found: " + regionName);
+    }
     LuceneQueryImpl<K, V> luceneQuery = new LuceneQueryImpl<K, V>(indexName, region, provider, projectionFields, limit, pageSize);
     return luceneQuery;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
index 07050e2..57b8862 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PartitionedRepositoryManager.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 
 import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.cache.Region;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 2d16e32..6429143 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -29,23 +29,26 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
-import java.util.function.Function;
 
 import com.gemstone.gemfire.cache.EvictionAttributes;
 import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.FixedPartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.lucene.LuceneIndex;
-import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
-import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -122,6 +125,18 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
+  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
+    createIndex("text");
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setOffHeap(true)
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      assertEquals(false, region.getOffHeap());
+    });
+  }
+
+  @Test
   public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
     createIndex("text");
     cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
@@ -131,18 +146,112 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldCreateInternalRegionsForIndex() {
+  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous() {
+    createIndex("text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(true)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue();
+    assertEquals(true, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldUseDiskSyncFalseOnQueueWhenUserRegionHasDiskSynchronousFalse() {
     createIndex("text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(false)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue();
+    assertEquals(false, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+    createIndex("field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    //Pause the sender so that the entry stays in the queue
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
+    queue.getSender().pause();
+
+    dataRegion.put("A", new TestObject());
+    cache.close();
+    createCache();
+    createIndex("field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+        "field1:world");
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      assertEquals(1, query.search().size());
+    });
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+    createIndex("field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    dataRegion.put("A", new TestObject());
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
+
+    //Wait until the queue has drained
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, queue.size()));
+    cache.close();
+    createCache();
+    createIndex("text");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+      "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  @Test
+  public void shouldCreateInternalRegionsForIndex() {
+    createIndex("field1", "field2");
 
     // Create partitioned region
     createRegion();
 
     verifyInternalRegions(region -> {
       region.isInternalRegion();
+      assertNotNull(region.getAttributes().getPartitionAttributes().getColocatedWith());
       cache.rootRegions().contains(region);
     });
   }
 
+  @Test
+  public void shouldUseFixedPartitionsForInternalRegions() {
+    createIndex("text");
+
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory<>();
+    final FixedPartitionAttributes fixedAttributes = FixedPartitionAttributes.createFixedPartition("A", true, 1);
+    partitionAttributesFactory.addFixedPartitionAttributes(fixedAttributes);
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setPartitionAttributes(partitionAttributesFactory.create())
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      //Fixed partitioned regions don't allow you to specify the partitions on the colocated region
+      assertNull(region.getAttributes().getPartitionAttributes().getFixedPartitionAttributes());
+      assertTrue(((PartitionedRegion) region).isFixedPartitionedRegion());
+    });
+  }
+
+
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
     // Get index
     LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
@@ -152,21 +261,25 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
     verify.accept(chunkRegion);
     verify.accept(fileRegion);
+  }
 
+  private AsyncEventQueue getIndexQueue() {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+    return cache.getAsyncEventQueue(aeqId);
   }
 
   private Region createRegion() {
     return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
   }
 
-  private void createIndex(String fieldName) {
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldName);
+  private void createIndex(String ... fieldNames) {
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
   }
 
   private static class TestObject implements Serializable {
 
-    String field1 = "a b c d";
-    String field2 = "f g h";
+    String field1 = "hello world";
+    String field2 = "this is a field";
   }
 
   private static class RecordingAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index 8c6f59e..67775d2 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -19,10 +19,15 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import java.io.File;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
 
 import org.junit.After;
 import org.junit.Before;
@@ -33,15 +38,15 @@ public class LuceneIntegrationTest {
 
   protected Cache cache;
   protected LuceneService luceneService;
-
   @Rule
-  public TestName name = new TestName();
+  public DiskDirRule diskDirRule = new DiskDirRule();
 
   @After
   public void tearDown() {
     if(this.cache != null) {
       this.cache.close();
     }
+    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Before
@@ -49,7 +54,12 @@ public class LuceneIntegrationTest {
     CacheFactory cf = new CacheFactory();
     cf.set("mcast-port", "0");
     cf.set("locators", "");
+    cf.set("off-heap-memory-size", "100m");
     this.cache = cf.create();
+    cache.createDiskStoreFactory()
+      .setDiskDirs(new File[] {diskDirRule.get()})
+      .setMaxOplogSize(1)
+      .create(GemFireCacheImpl.getDefaultDiskStoreName());
     luceneService = LuceneServiceProvider.get(this.cache);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
index 984b221..e35de5e 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.ExpirationAttributes;
 import com.gemstone.gemfire.cache.MembershipAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl;
@@ -120,12 +121,16 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
+    final Region region =Fakes.region(regionPath, cache);
+    RegionAttributes attributes  = region.getAttributes();
+    when(attributes.getDataPolicy()).thenReturn(DataPolicy.PERSISTENT_PARTITION);
     AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
     when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.createAEQWithPersistence();
+    index.createAEQ(region);
 
+    verify(aeqFactory).setPersistent(eq(true));
     verify(aeqFactory).create(any(), any());
   }
 
@@ -134,21 +139,23 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
+    final Region region =Fakes.region(regionPath, cache);
     AsyncEventQueueFactoryImpl aeqFactory = mock(AsyncEventQueueFactoryImpl.class);
     when(cache.createAsyncEventQueueFactory()).thenReturn(aeqFactory);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.createAEQ();
+    index.createAEQ(region);
 
+    verify(aeqFactory, never()).setPersistent(eq(true));
     verify(aeqFactory).create(any(), any());
   }
 
-  private void initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache) {
+  private Region initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache) {
     int defaultLocalMemory = 100;
-    initializeScenario(withPersistence, regionPath, cache, defaultLocalMemory);
+    return initializeScenario(withPersistence, regionPath, cache, defaultLocalMemory);
   }
 
-  private void initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache, int localMaxMemory) {
+  private Region initializeScenario(final boolean withPersistence, final String regionPath, final Cache cache, int localMaxMemory) {
     PartitionedRegion region = mock(PartitionedRegion.class);
     RegionAttributes regionAttributes = mock(RegionAttributes.class);
     PartitionAttributes partitionAttributes = mock(PartitionAttributes.class);
@@ -162,6 +169,8 @@ public class LuceneIndexForPartitionedRegionTest {
     when(partitionAttributes.getTotalNumBuckets()).thenReturn(113);
     when(dataPolicy.withPersistence()).thenReturn(withPersistence);
     when(region.getExtensionPoint()).thenReturn(extensionPoint);
+
+    return region;
   }
 
   private PartitionAttributes initializeAttributes(final Cache cache) {
@@ -192,41 +201,22 @@ public class LuceneIndexForPartitionedRegionTest {
   }
 
   @Test
-  public void initializeWithPersistenceShouldCreateAEQWithPersistence() {
-    boolean withPersistence = true;
-    String name = "indexName";
-    String regionPath = "regionName";
-    Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
-
-    LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.setSearchableFields(new String[]{"field"});
-    LuceneIndexForPartitionedRegion spy = spy(index);
-    doReturn(null).when(spy).createFileRegion(any(), any(), any());
-    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
-    spy.initialize();
-
-    verify(spy).createAEQWithPersistence();
-  }
-
-  @Test
   public void initializeWithoutPersistenceShouldCreateAEQ() {
     boolean withPersistence = false;
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
+    Region region = initializeScenario(withPersistence, regionPath, cache);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[]{"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ();
+    doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
 
-    verify(spy).createAEQ();
+    verify(spy).createAEQ(eq(region));
   }
 
   @Test
@@ -235,14 +225,14 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
+    Region region = initializeScenario(withPersistence, regionPath, cache);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[]{"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ();
+    doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
 
     verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()));
@@ -254,14 +244,14 @@ public class LuceneIndexForPartitionedRegionTest {
     String name = "indexName";
     String regionPath = "regionName";
     Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
+    Region region = initializeScenario(withPersistence, regionPath, cache);
 
     LuceneIndexForPartitionedRegion index = new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[]{"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ();
+    doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()), any());
@@ -309,7 +299,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
+    doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
 
     verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()));
@@ -328,7 +318,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
+    doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION_PERSISTENT), eq(index.createFileRegionName()), any());
@@ -347,7 +337,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = spy(index);
     doReturn(null).when(spy).createFileRegion(any(), any(), any());
     doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQWithPersistence();
+    doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
     spy.initialize();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7908467a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
index 0614e62..975b92f 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryFactoryImplJUnitTest.java
@@ -19,21 +19,30 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
 
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class LuceneQueryFactoryImplJUnitTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
 
   @Test
-  public void test() {
-    Cache cache = Mockito.mock(Cache.class);
+  public void shouldCreateQueryWithCorrectAttributes() {
+    Cache cache = mock(Cache.class);
+    Region region = mock(Region.class);
+    when(cache.getRegion(any())).thenReturn(region);
     LuceneQueryFactoryImpl f = new LuceneQueryFactoryImpl(cache);
     f.setPageSize(5);
     f.setResultLimit(25);
@@ -47,4 +56,12 @@ public class LuceneQueryFactoryImplJUnitTest {
     Mockito.verify(cache).getRegion(Mockito.eq("region"));
   }
 
+  @Test
+  public void shouldThrowIllegalArgumentWhenRegionIsMissing() {
+    Cache cache = mock(Cache.class);
+    LuceneQueryFactoryImpl f = new LuceneQueryFactoryImpl(cache);
+    thrown.expect(IllegalArgumentException.class);
+    LuceneQuery<Object, Object> query = f.create("index", "region", new StringQueryProvider("test"));
+  }
+
 }


[23/50] [abbrv] incubator-geode git commit: GEODE-11: Split out lucene tests of persistence and off heap

Posted by kl...@apache.org.
GEODE-11: Split out lucene tests of persistence and off heap

Persistence and offheap require some special before and after code to
create and clean up the resources. Splitting these to separate
integration tests.


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

Branch: refs/heads/feature/GEODE-835
Commit: 8024f33d5687ebbbfda7a0e20375620ff58015d9
Parents: 09b3c03
Author: Dan Smith <up...@apache.org>
Authored: Fri May 13 09:51:43 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue May 17 12:11:40 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexCreationIntegrationTest.java     | 159 ++++---------------
 ...ceneIndexCreationOffHeapIntegrationTest.java |  72 +++++++++
 ...IndexCreationPersistenceIntegrationTest.java | 141 ++++++++++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |  20 ++-
 .../cache/lucene/test/LuceneTestUtilities.java  |  64 ++++++++
 .../gemfire/cache/lucene/test/TestObject.java   |  27 ++++
 .../gemfire/cache/lucene/test/package-info.java |  23 +++
 7 files changed, 365 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index ef7f808..4c28938 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -19,9 +19,9 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -36,10 +36,8 @@ import com.gemstone.gemfire.cache.FixedPartitionAttributes;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
-import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
@@ -47,14 +45,26 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+/**
+ * Tests of creating lucene indexes on regions. All tests of index creation
+ * use cases should be in classes starting with LuceneIndexCreation*. Most
+ * tests belong in this class, except for:
+ * <ul>
+ * <li> Tests that use persistence are in {@link LuceneIndexCreationPersistenceIntegrationTest}  </li>
+ * <li> Tests that use offheap are in {@link LuceneIndexCreationOffHeapIntegrationTest}  </li>
+ * </ul>
+ */
 @Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
-  public static final String INDEX_NAME = "index";
-  public static final String REGION_NAME = "region";
+
 
   @Test
   public void shouldCreateIndexWriterWithAnalyzersWhenSettingPerFieldAnalyzers()
@@ -70,18 +80,22 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     Region region = createRegion();
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     region.put("key1", new TestObject());
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
     assertEquals(analyzers, index.getFieldAnalyzers());
     assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
     assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
   }
 
   @Test
-  public void shouldUseRedundancyForInternalRegionsWhenUserRegionHasRedundancy() {
+  @Parameters({"0", "1", "2"})
+  public void shouldUseRedundancyForInternalRegionsWhenUserRegionHasRedundancy(int redundancy) {
     createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_REDUNDANT).create(REGION_NAME);
+    PartitionAttributesFactory paf = new PartitionAttributesFactory();
+    paf.setRedundantCopies(redundancy);
+
+    cache.createRegionFactory(RegionShortcut.PARTITION_REDUNDANT).setPartitionAttributes(paf.create()).create(REGION_NAME);
     verifyInternalRegions(region -> {
-      assertEquals(1, region.getAttributes().getPartitionAttributes().getRedundantCopies());
+      assertEquals(redundancy, region.getAttributes().getPartitionAttributes().getRedundantCopies());
     });
   }
 
@@ -122,103 +136,6 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION)
-      .setOffHeap(true)
-      .create(REGION_NAME);
-
-    verifyInternalRegions(region -> {
-      assertEquals(false, region.getOffHeap());
-    });
-  }
-
-  @Test
-  public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getAttributes().getEvictionAttributes().getAction().isNone());
-    });
-  }
-
-  @Test
-  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .setDiskSynchronous(true)
-      .create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getDataPolicy().withPersistence());
-      assertTrue(region.isDiskSynchronous());
-    });
-    AsyncEventQueue queue = getIndexQueue();
-    assertEquals(true, queue.isDiskSynchronous());
-    assertEquals(true, queue.isPersistent());
-  }
-
-  @Test
-  public void shouldUseDiskSyncFalseOnQueueWhenUserRegionHasDiskSynchronousFalse() {
-    createIndex("text");
-    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .setDiskSynchronous(false)
-      .create(REGION_NAME);
-    verifyInternalRegions(region -> {
-      assertTrue(region.getDataPolicy().withPersistence());
-      assertTrue(region.isDiskSynchronous());
-    });
-    AsyncEventQueue queue = getIndexQueue();
-    assertEquals(false, queue.isDiskSynchronous());
-    assertEquals(true, queue.isPersistent());
-  }
-
-  @Test
-  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
-    createIndex("field1", "field2");
-    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    //Pause the sender so that the entry stays in the queue
-    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
-    queue.getSender().pause();
-
-    dataRegion.put("A", new TestObject());
-    cache.close();
-    createCache();
-    createIndex("field1", "field2");
-    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
-    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
-      .create(INDEX_NAME, REGION_NAME,
-        "field1:world");
-    assertEquals(1, query.search().size());
-  }
-
-  private void verifyIndexFinishFlushing(String indexName, String regionName) {
-    LuceneIndex index = luceneService.getIndex(indexName, regionName);
-    boolean flushed = index.waitUntilFlushed(60000);
-    assertTrue(flushed);
-  }
-  
-  @Test
-  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
-    createIndex("field1", "field2");
-    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    dataRegion.put("A", new TestObject());
-    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
-    cache.close();
-    createCache();
-    createIndex("text");
-    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
-      .create(REGION_NAME);
-    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
-      .create(INDEX_NAME, REGION_NAME,
-      "field1:world");
-    assertEquals(1, query.search().size());
-  }
-
-  @Test
   public void shouldCreateInternalRegionsForIndex() {
     createIndex("field1", "field2");
 
@@ -252,19 +169,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
 
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
-    // Get index
-    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
-
-    // Verify the meta regions exist and are internal
-    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
-    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
-    verify.accept(chunkRegion);
-    verify.accept(fileRegion);
-  }
-
-  private AsyncEventQueue getIndexQueue() {
-    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
-    return cache.getAsyncEventQueue(aeqId);
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
 
   private Region createRegion() {
@@ -272,13 +177,7 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   private void createIndex(String ... fieldNames) {
-    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
-  }
-
-  private static class TestObject implements Serializable {
-
-    String field1 = "hello world";
-    String field2 = "this is a field";
+    LuceneTestUtilities.createIndex(cache, fieldNames);
   }
 
   private static class RecordingAnalyzer extends Analyzer {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
new file mode 100644
index 0000000..8f10c26
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationOffHeapIntegrationTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.cache.lucene;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.assertEquals;
+
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.offheap.MemoryAllocatorImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests of lucene index creation that use off heap memory
+ */
+@Category(IntegrationTest.class)
+public class LuceneIndexCreationOffHeapIntegrationTest extends LuceneIntegrationTest {
+
+  @Override
+  public void closeCache() {
+    super.closeCache();
+    MemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Override
+  protected CacheFactory getCacheFactory() {
+    CacheFactory factory = super.getCacheFactory();
+    factory.set("off-heap-memory-size", "100m");
+    return factory;
+  }
+
+  @Test
+  public void shouldNotUseOffHeapForInternalRegionsWhenUserRegionHasOffHeap() {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION)
+      .setOffHeap(true)
+      .create(REGION_NAME);
+
+    verifyInternalRegions(region -> {
+      assertEquals(false, region.getOffHeap());
+    });
+  }
+
+  private void verifyInternalRegions(Consumer<LocalRegion> verify) {
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
new file mode 100644
index 0000000..23983cb
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.cache.lucene;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
+import com.jayway.awaitility.Awaitility;
+
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+/**
+ * Tests of lucene index creation that use persistence
+ */
+@Category(IntegrationTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegrationTest {
+
+  @Rule
+  public DiskDirRule diskDirRule = new DiskDirRule();
+
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  @Override
+  public void createCache() {
+    super.createCache();
+    cache.createDiskStoreFactory()
+      .setDiskDirs(new File[] {diskDirRule.get()})
+      .setMaxOplogSize(1)
+      .create(GemFireCacheImpl.getDefaultDiskStoreName());
+  }
+
+  @Test
+  public void shouldNotUseOverflowForInternalRegionsWhenUserRegionHasOverflow() {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_OVERFLOW).create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getAttributes().getEvictionAttributes().getAction().isNone());
+    });
+  }
+
+  @Test
+  @Parameters({"true", "false"})
+  public void shouldUseDiskSynchronousWhenUserRegionHasDiskSynchronous(boolean synchronous) {
+    createIndex(cache, "text");
+    cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .setDiskSynchronous(synchronous)
+      .create(REGION_NAME);
+    verifyInternalRegions(region -> {
+      assertTrue(region.getDataPolicy().withPersistence());
+      //Underlying region should always be synchronous
+      assertTrue(region.isDiskSynchronous());
+    });
+    AsyncEventQueue queue = getIndexQueue(cache);
+    assertEquals(synchronous, queue.isDiskSynchronous());
+    assertEquals(true, queue.isPersistent());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+    createIndex(cache, "field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    //Pause the sender so that the entry stays in the queue
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue(cache);
+    queue.getSender().pause();
+
+    dataRegion.put("A", new TestObject());
+    cache.close();
+    createCache();
+    createIndex(cache, "field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+        "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  @Test
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+    createIndex(cache, "field1", "field2");
+    Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    dataRegion.put("A", new TestObject());
+    verifyIndexFinishFlushing(cache, INDEX_NAME, REGION_NAME);
+    cache.close();
+    createCache();
+    createIndex(cache, "field1", "field2");
+    dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
+      .create(REGION_NAME);
+    LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .create(INDEX_NAME, REGION_NAME,
+      "field1:world");
+    assertEquals(1, query.search().size());
+  }
+
+  private void verifyInternalRegions(Consumer<LocalRegion> verify) {
+    LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index 67775d2..c302460 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -38,28 +38,26 @@ public class LuceneIntegrationTest {
 
   protected Cache cache;
   protected LuceneService luceneService;
-  @Rule
-  public DiskDirRule diskDirRule = new DiskDirRule();
 
   @After
-  public void tearDown() {
+  public void closeCache() {
     if(this.cache != null) {
       this.cache.close();
     }
-    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 
   @Before
   public void createCache() {
+    CacheFactory cf = getCacheFactory();
+    this.cache = cf.create();
+
+    luceneService = LuceneServiceProvider.get(this.cache);
+  }
+
+  protected CacheFactory getCacheFactory() {
     CacheFactory cf = new CacheFactory();
     cf.set("mcast-port", "0");
     cf.set("locators", "");
-    cf.set("off-heap-memory-size", "100m");
-    this.cache = cf.create();
-    cache.createDiskStoreFactory()
-      .setDiskDirs(new File[] {diskDirRule.get()})
-      .setMaxOplogSize(1)
-      .create(GemFireCacheImpl.getDefaultDiskStoreName());
-    luceneService = LuceneServiceProvider.get(this.cache);
+    return cf;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
new file mode 100644
index 0000000..61355c4
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -0,0 +1,64 @@
+/*
+ * 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.cache.lucene.test;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.function.Consumer;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+
+public class LuceneTestUtilities {
+  public static final String INDEX_NAME = "index";
+  public static final String REGION_NAME = "region";
+
+  public static void verifyInternalRegions(LuceneService luceneService, Cache cache, Consumer<LocalRegion> verify) {
+    // Get index
+    LuceneIndexForPartitionedRegion index = (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    // Verify the meta regions exist and are internal
+    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
+    LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
+    verify.accept(chunkRegion);
+    verify.accept(fileRegion);
+  }
+
+  public static AsyncEventQueue getIndexQueue(Cache cache) {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
+    return cache.getAsyncEventQueue(aeqId);
+  }
+
+  public static void createIndex(Cache cache, String... fieldNames) {
+    LuceneServiceProvider.get(cache).createIndex(INDEX_NAME, REGION_NAME, fieldNames);
+  }
+
+  public static void verifyIndexFinishFlushing(Cache cache, String indexName, String regionName) {
+    LuceneService luceneService = LuceneServiceProvider.get(cache);
+    LuceneIndex index = luceneService.getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
new file mode 100644
index 0000000..3287b9f
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cache.lucene.test;
+
+import java.io.Serializable;
+
+public class TestObject implements Serializable {
+
+  String field1 = "hello world";
+  String field2 = "this is a field";
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8024f33d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
new file mode 100644
index 0000000..6345804
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+/**
+ * Utility classes for testing the lucene functionality.
+ */
+
+package com.gemstone.gemfire.cache.lucene.test;
\ No newline at end of file


[21/50] [abbrv] incubator-geode git commit: GEODE-1256 Alter rat.gradle to exclude copies of website sources

Posted by kl...@apache.org.
GEODE-1256  Alter rat.gradle to exclude copies of website sources

  Added `geode-site/content/**` to the rat.gradle excludes list
  such that there are no longer 10 Unknown Licenses.

This closes #144


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

Branch: refs/heads/feature/GEODE-835
Commit: 3e181e83d4177ac4acf7412430d820a11f81c811
Parents: ab417e3
Author: Karen Miller <km...@pivotal.io>
Authored: Mon May 16 18:22:01 2016 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Tue May 17 10:24:55 2016 -0700

----------------------------------------------------------------------
 gradle/rat.gradle | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e181e83/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 3d0826a..f507993 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -117,6 +117,7 @@ rat {
     'geode-site/website/content/css/font-awesome.min.css',
     'geode-site/website/lib/pandoc.template',
     'geode-site/website/content/font/**',
+    'geode-site/content/**',
     // compiled logs and locks
     'geode-site/website/tmp/**',
     'geode-site/website/layouts/**',


[43/50] [abbrv] incubator-geode git commit: GEODE-1412: ignore-eviction-expiration flag was not used in the http controller

Posted by kl...@apache.org.
GEODE-1412: ignore-eviction-expiration flag was not used in the http controller


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

Branch: refs/heads/feature/GEODE-835
Commit: 3a1c1a964b6c00098cdf95b2f9af596b00ec6455
Parents: f272cf0
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu May 19 10:12:51 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu May 19 19:32:47 2016 -0700

----------------------------------------------------------------------
 .../internal/web/controllers/QueueCommandsController.java           | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3a1c1a96/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 83d7bc1..519b5b0 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
@@ -83,6 +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));
 
     if (hasValue(batchSize)) {
       command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE, String.valueOf(batchSize));


[38/50] [abbrv] incubator-geode git commit: GEODE-17: clean up error messages

Posted by kl...@apache.org.
GEODE-17: clean up error messages

* clean up authentication/authorization error messages
* Catch Authorization exception later in the command chain to avoid unnecesary parsing of command result
* Add ExceptionHandler in controller to set the http header correctly
* Catch Authorization exception in gfsh execution for better error report


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

Branch: refs/heads/feature/GEODE-835
Commit: 14a548ff22d7055f1da75e5432412472113e5e9d
Parents: 21c0e24
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue May 17 14:48:30 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu May 19 10:40:57 2016 -0700

----------------------------------------------------------------------
 .../internal/security/GeodeSecurityUtil.java    |  55 ++---
 .../internal/beans/MemberMBeanBridge.java       |  18 +-
 .../internal/cli/remote/CommandProcessor.java   |   9 +-
 .../internal/cli/result/ResultBuilder.java      |   3 +-
 .../cli/shell/GfshExecutionStrategy.java        | 230 ++++++++++---------
 .../security/ResourceOperationContext.java      |   2 +-
 .../controllers/AbstractCommandsController.java |  88 ++-----
 .../web/shell/AbstractHttpOperationInvoker.java |  16 +-
 .../security/GfshCommandsSecurityTest.java      |  11 +-
 .../security/MemberMBeanSecurityJUnitTest.java  |   2 +-
 .../WanCommandsControllerJUnitTest.java         |   7 +-
 11 files changed, 199 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 236b00b..322c59e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 import com.gemstone.gemfire.management.internal.security.ResourceOperationContext;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
@@ -53,31 +54,6 @@ public class GeodeSecurityUtil {
   private static Logger logger = LogService.getLogger();
 
   /**
-   *
-   * @param username
-   * @param password
-   * @return null if security is not enabled, otherwise return a shiro subject
-   */
-  public static Subject login(String username, String password){
-    if(!isSecured())
-      return null;
-
-    Subject currentUser = SecurityUtils.getSubject();
-
-    UsernamePasswordToken token =
-        new UsernamePasswordToken(username, password);
-    try {
-      logger.info("Logging in "+username+"/"+password);
-      currentUser.login(token);
-    } catch (ShiroException e) {
-      logger.info(e.getMessage(), e);
-      throw new AuthenticationFailedException(e.getMessage(), e);
-    }
-
-    return currentUser;
-  }
-
-  /**
    * It first looks the shiro subject in AccessControlContext since JMX will use multiple threads to process operations from the same client.
    * then it looks into Shiro's thead context.
    *
@@ -114,6 +90,31 @@ public class GeodeSecurityUtil {
     return currentUser;
   }
 
+  /**
+   *
+   * @param username
+   * @param password
+   * @return null if security is not enabled, otherwise return a shiro subject
+   */
+  public static Subject login(String username, String password){
+    if(!isSecured())
+      return null;
+
+    Subject currentUser = SecurityUtils.getSubject();
+
+    UsernamePasswordToken token =
+      new UsernamePasswordToken(username, password);
+    try {
+      logger.info("Logging in "+username+"/"+password);
+      currentUser.login(token);
+    } catch (ShiroException e) {
+      logger.info(e.getMessage(), e);
+      throw new AuthenticationFailedException("Authentication error. Please check your username/password.", e);
+    }
+
+    return currentUser;
+  }
+
   public static void logout(){
     Subject currentUser = getSubject();
     if(currentUser==null)
@@ -125,7 +126,7 @@ public class GeodeSecurityUtil {
     }
     catch(ShiroException e){
       logger.info(e.getMessage(), e);
-      throw new AuthenticationFailedException(e.getMessage(), e);
+      throw new GemFireSecurityException(e.getMessage(), e);
     }
     // clean out Shiro's thread local content
     ThreadContext.remove();
@@ -205,7 +206,7 @@ public class GeodeSecurityUtil {
     }
     catch(ShiroException e){
       logger.info(currentUser.getPrincipal() + " not authorized for " + context);
-      throw new GemFireSecurityException(e.getMessage(), e);
+      throw new NotAuthorizedException(e.getMessage(), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
index 67ad60d..98258e8 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
@@ -123,10 +123,9 @@ import com.gemstone.gemfire.management.internal.cli.CommandResponseBuilder;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandExecutionContext;
 import com.gemstone.gemfire.management.internal.cli.remote.MemberCommandService;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
-import com.gemstone.gemfire.management.internal.cli.result.ErrorResultData;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
-import com.gemstone.gemfire.security.GemFireSecurityException;
+
 import org.apache.logging.log4j.Logger;
 
 /**
@@ -1738,17 +1737,8 @@ public class MemberMBeanBridge {
     }
 
     if (isGfshRequest) {
-      String responseJson = CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
-  //    System.out.println("responseJson :: "+responseJson);
-      return responseJson;
+      return CommandResponseBuilder.createCommandResponseJson(getMember(), (CommandResult) result);
     } else {
-      // throw GemFireSecurityException is the returned error code is 415
-      if(((CommandResult) result).getResultData() instanceof ErrorResultData){
-        ErrorResultData resultData = (ErrorResultData) ((CommandResult)result).getResultData();
-        if(resultData.getErrorCode()==ResultBuilder.ERRORCODE_UNAUTHORIZED){
-          throw new GemFireSecurityException(resultData.getGfJsonObject().toString());
-        }
-      }
       return ResultBuilder.resultAsString(result);
     }
   }
@@ -1758,14 +1748,12 @@ public class MemberMBeanBridge {
     if (env != null) {
       appName = env.get(Gfsh.ENV_APP_NAME);
     }
-//    System.out.println("appName :: "+appName);
     
     return Gfsh.GFSH_APP_NAME.equals(appName);
   }
   
   public long getTotalDiskUsage() {
-    long diskSpaceUsage = regionMonitor.getDiskSpace();
-    return diskSpaceUsage;
+    return regionMonitor.getDiskSpace();
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
index 7edc3e4..69bc64e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
@@ -21,6 +21,7 @@ import java.lang.reflect.Method;
 import java.util.Map;
 import java.util.Properties;
 
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.cli.CommandProcessingException;
 import com.gemstone.gemfire.management.cli.CommandStatement;
 import com.gemstone.gemfire.management.cli.Result;
@@ -30,8 +31,7 @@ import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.util.CommentSkipHelper;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
-import com.gemstone.gemfire.security.GemFireSecurityException;
-import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.springframework.shell.core.Parser;
 import org.springframework.shell.event.ParseResult;
@@ -126,12 +126,13 @@ public class CommandProcessor {
           logWrapper.info("Could not parse \""+cmdStmt.getCommandString()+"\".", e);
         }
         return ResultBuilder.createParsingErrorResult(e.getMessage());
-      } catch (GemFireSecurityException e) {
+      } catch (NotAuthorizedException e) {
         setLastExecutionStatus(1);
         if (logWrapper.infoEnabled()) {
           logWrapper.info("Could not execute \""+cmdStmt.getCommandString()+"\".", e);
         }
-        return ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized while processing command <" +cmdStmt.getCommandString()+"> Reason : " + e.getMessage());
+        // for NotAuthorizedException, will catch this later in the code
+        throw e;
       }catch (RuntimeException e) {
         setLastExecutionStatus(1);
         if (logWrapper.infoEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
index 6b435d3..55f7a89 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/result/ResultBuilder.java
@@ -116,8 +116,7 @@ public class ResultBuilder {
   }
 
   public static Result createGemFireUnAuthorizedErrorResult(String message) {
-    return createErrorResult(ERRORCODE_UNAUTHORIZED,
-        "Could not process command due to GemFire error. " + message);
+    return createErrorResult(ERRORCODE_UNAUTHORIZED, message);
   }
 
   public static Result createUserErrorResult(String message) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
index c5ebe9a..0cfae9c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/GfshExecutionStrategy.java
@@ -16,17 +16,11 @@
  */
 package com.gemstone.gemfire.management.internal.cli.shell;
 
-import static com.gemstone.gemfire.management.internal.cli.multistep.CLIMultiStepHelper.execCLISteps;
+import static com.gemstone.gemfire.management.internal.cli.multistep.CLIMultiStepHelper.*;
 
 import java.lang.reflect.Method;
 import java.util.Map;
 
-import org.springframework.shell.core.ExecutionStrategy;
-import org.springframework.shell.core.Shell;
-import org.springframework.shell.event.ParseResult;
-import org.springframework.util.Assert;
-import org.springframework.util.ReflectionUtils;
-
 import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.management.cli.CliMetaData;
 import com.gemstone.gemfire.management.cli.CommandProcessingException;
@@ -42,6 +36,13 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.multistep.MultiStepCommand;
 import com.gemstone.gemfire.management.internal.cli.result.FileResult;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
+import com.gemstone.gemfire.security.NotAuthorizedException;
+
+import org.springframework.shell.core.ExecutionStrategy;
+import org.springframework.shell.core.Shell;
+import org.springframework.shell.event.ParseResult;
+import org.springframework.util.Assert;
+import org.springframework.util.ReflectionUtils;
 
 /**
  * Defines the {@link ExecutionStrategy} for commands that are executed in
@@ -81,43 +82,40 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
     Object result = null;
     Method method = parseResult.getMethod();
     try {
-      
-    //Check if it's a multi-step command
+      //Check if it's a multi-step command
       Method reflectmethod = parseResult.getMethod();
-      MultiStepCommand cmd = reflectmethod.getAnnotation(MultiStepCommand.class);      
-      if(cmd!=null){
-        return execCLISteps(logWrapper, shell,parseResult);
+      MultiStepCommand cmd = reflectmethod.getAnnotation(MultiStepCommand.class);
+      if (cmd != null) {
+        return execCLISteps(logWrapper, shell, parseResult);
       }
 
-//      See #46072 
-//      String commandName = getCommandName(parseResult);
-//      if (commandName != null) {
-//        shell.flashMessage("Executing " + getCommandName(parseResult) + " ... ");
-//      }
-      //Check if it's a remote command
-      if (!isShellOnly(method)) {
-        if (GfshParseResult.class.isInstance(parseResult)) {
-          result = executeOnRemote((GfshParseResult)parseResult);
-        } else {//Remote command means implemented for Gfsh and ParseResult should be GfshParseResult.
-          //TODO - Abhishek: should this message be more specific?
-          throw new IllegalStateException("Configuration error!");
-        }
-      } else {
+      //check if it's a shell only command
+      if(isShellOnly(method)){
         Assert.notNull(parseResult, "Parse result required");
         synchronized (mutex) {
-          //TODO: Remove Assert
           Assert.isTrue(isReadyForCommands(), "ProcessManagerHostedExecutionStrategy not yet ready for commands");
-          result = ReflectionUtils.invokeMethod(parseResult.getMethod(), parseResult.getInstance(), parseResult.getArguments());
+          return ReflectionUtils.invokeMethod(parseResult.getMethod(), parseResult.getInstance(), parseResult.getArguments());
         }
       }
-//    See #46072
-//      shell.flashMessage("");
-    } catch (JMXInvocationException e) {
+
+      //check if it's a GfshParseResult
+      if(!GfshParseResult.class.isInstance(parseResult)){
+        throw new IllegalStateException("Configuration error!");
+      }
+
+      result = executeOnRemote((GfshParseResult) parseResult);
+    }
+    catch(NotAuthorizedException e) {
+      result = ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized. Reason : " + e.getMessage());
+    }
+    catch (JMXInvocationException e) {
       Gfsh.getCurrentInstance().logWarning(e.getMessage(), e);
-    } catch (IllegalStateException e) {
+    }
+    catch (IllegalStateException e) {
       // Shouldn't occur - we are always using GfsParseResult
       Gfsh.getCurrentInstance().logWarning(e.getMessage(), e);
-    } catch (CommandProcessingException e) {
+    }
+    catch (CommandProcessingException e) {
       Gfsh.getCurrentInstance().logWarning(e.getMessage(), null);
       Object errorData = e.getErrorData();
       if (errorData != null && errorData instanceof Throwable) {
@@ -125,16 +123,17 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
       } else {
         logWrapper.warning(e.getMessage());
       }
-    } catch (RuntimeException e) {
+    }
+    catch (RuntimeException e) {
       Gfsh.getCurrentInstance().logWarning("Exception occurred. " + e.getMessage(), e);
       // Log other runtime exception in gfsh log
       logWrapper.warning("Error occurred while executing command : "+((GfshParseResult)parseResult).getUserInput(), e);
-    } catch (Exception e) {
+    }
+    catch (Exception e) {
       Gfsh.getCurrentInstance().logWarning("Unexpected exception occurred. " + e.getMessage(), e);
       // Log other exceptions in gfsh log
       logWrapper.warning("Unexpected error occurred while executing command : "+((GfshParseResult)parseResult).getUserInput(), e);
     }
-    
     return result;
   }
   
@@ -204,90 +203,97 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
   private Result executeOnRemote(GfshParseResult parseResult) {
     Result   commandResult = null;
     Object   response      = null;
-    if (shell.isConnectedAndReady()) {
-      byte[][]             fileData    = null;
-      CliAroundInterceptor interceptor = null;
-      
-      String interceptorClass = getInterceptor(parseResult.getMethod());
-      
-      //1. Pre Remote Execution
-      if (!CliMetaData.ANNOTATION_NULL_VALUE.equals(interceptorClass)) {
-        try {
-          interceptor = (CliAroundInterceptor) ClassPathLoader.getLatest().forName(interceptorClass).newInstance();
-        } catch (InstantiationException e) {
-          shell.logWarning("Configuration error", e);
-        } catch (IllegalAccessException e) {
-          shell.logWarning("Configuration error", e);
-        } catch (ClassNotFoundException e) {
-          shell.logWarning("Configuration error", e);
-        }
-        if (interceptor != null) {
-          Result preExecResult = interceptor.preExecution(parseResult);
-          if (Status.ERROR.equals(preExecResult.getStatus())) {
-            return preExecResult;
-          } else if (preExecResult instanceof FileResult) {            
-            FileResult fileResult = (FileResult) preExecResult;
-            fileData = fileResult.toBytes();
-          }
-        } else {
-          return ResultBuilder.createBadConfigurationErrorResult("Interceptor Configuration Error");
+
+    if(!shell.isConnectedAndReady()){
+      shell.logWarning("Can't execute a remote command without connection. Use 'connect' first to connect.", null);
+      logWrapper.info("Can't execute a remote command \""+parseResult.getUserInput()+"\" without connection. Use 'connect' first to connect to GemFire.");
+      return null;
+    }
+
+    byte[][]             fileData    = null;
+    CliAroundInterceptor interceptor = null;
+
+    String interceptorClass = getInterceptor(parseResult.getMethod());
+
+    //1. Pre Remote Execution
+    if (!CliMetaData.ANNOTATION_NULL_VALUE.equals(interceptorClass)) {
+      try {
+        interceptor = (CliAroundInterceptor) ClassPathLoader.getLatest().forName(interceptorClass).newInstance();
+      } catch (InstantiationException e) {
+        shell.logWarning("Configuration error", e);
+      } catch (IllegalAccessException e) {
+        shell.logWarning("Configuration error", e);
+      } catch (ClassNotFoundException e) {
+        shell.logWarning("Configuration error", e);
+      }
+      if (interceptor != null) {
+        Result preExecResult = interceptor.preExecution(parseResult);
+        if (Status.ERROR.equals(preExecResult.getStatus())) {
+          return preExecResult;
+        } else if (preExecResult instanceof FileResult) {
+          FileResult fileResult = (FileResult) preExecResult;
+          fileData = fileResult.toBytes();
         }
+      } else {
+        return ResultBuilder.createBadConfigurationErrorResult("Interceptor Configuration Error");
       }
+    }
 
-      //2. Remote Execution
-      final Map<String, String> env = shell.getEnv();
+    //2. Remote Execution
+    final Map<String, String> env = shell.getEnv();
+    try {
       response = shell.getOperationInvoker().processCommand(new CommandRequest(parseResult, env, fileData));
+    } catch(NotAuthorizedException e) {
+      return ResultBuilder.createGemFireUnAuthorizedErrorResult("Unauthorized. Reason : " + e.getMessage());
+    }
+    finally {
       env.clear();
-      
-      if (response == null) {
-        shell.logWarning("Response was null for: \""+parseResult.getUserInput()+"\". (gfsh.isConnected="+shell.isConnectedAndReady()+")", null);
-        commandResult = 
-            ResultBuilder.createBadResponseErrorResult(" Error occurred while " + 
-                "executing \""+parseResult.getUserInput()+"\" on manager. " +
-                		"Please check manager logs for error.");
-      } else {
-        if (logWrapper.fineEnabled()) {
-          logWrapper.fine("Received response :: "+response);
-        }
-        CommandResponse commandResponse = CommandResponseBuilder.prepareCommandResponseFromJson((String) response);
-        
-        if (commandResponse.isFailedToPersist()) {
-          shell.printAsSevere(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
-          logWrapper.severe(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
-        }
-        
-        String debugInfo = commandResponse.getDebugInfo();
-        if (debugInfo != null && !debugInfo.trim().isEmpty()) {
-          //TODO - Abhishek When debug is ON, log response in gfsh logs
-          //TODO - Abhishek handle \n better. Is it coming from GemFire formatter
-          debugInfo = debugInfo.replaceAll("\n\n\n", "\n");
-          debugInfo = debugInfo.replaceAll("\n\n", "\n");
-          debugInfo = debugInfo.replaceAll("\n", "\n[From Manager : "+commandResponse.getSender()+"]");
-          debugInfo = "[From Manager : "+commandResponse.getSender()+"]" + debugInfo;
-          LogWrapper.getInstance().info(debugInfo);
-        }
-        commandResult = ResultBuilder.fromJson((String) response);
-        
-        
-        //3. Post Remote Execution
-        if (interceptor != null) {
-          Result postExecResult = interceptor.postExecution(parseResult, commandResult);
-          if (postExecResult != null) {
-            if (Status.ERROR.equals(postExecResult.getStatus())) {
-              if (logWrapper.infoEnabled()) {
-                logWrapper.info("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
-              }
-            } else if (logWrapper.fineEnabled()) {
-              logWrapper.fine("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
-            }
-            commandResult = postExecResult;
+    }
+
+    if (response == null) {
+      shell.logWarning("Response was null for: \"" + parseResult.getUserInput() + "\". (gfsh.isConnected=" + shell.isConnectedAndReady() + ")", null);
+      return ResultBuilder.createBadResponseErrorResult(" Error occurred while " +
+        "executing \"" + parseResult.getUserInput() + "\" on manager. " +
+        "Please check manager logs for error.");
+    }
+
+    if (logWrapper.fineEnabled()) {
+      logWrapper.fine("Received response :: "+response);
+    }
+    CommandResponse commandResponse = CommandResponseBuilder.prepareCommandResponseFromJson((String) response);
+
+    if (commandResponse.isFailedToPersist()) {
+      shell.printAsSevere(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
+      logWrapper.severe(CliStrings.SHARED_CONFIGURATION_FAILED_TO_PERSIST_COMMAND_CHANGES);
+    }
+
+    String debugInfo = commandResponse.getDebugInfo();
+    if (debugInfo != null && !debugInfo.trim().isEmpty()) {
+      //TODO - Abhishek When debug is ON, log response in gfsh logs
+      //TODO - Abhishek handle \n better. Is it coming from GemFire formatter
+      debugInfo = debugInfo.replaceAll("\n\n\n", "\n");
+      debugInfo = debugInfo.replaceAll("\n\n", "\n");
+      debugInfo = debugInfo.replaceAll("\n", "\n[From Manager : "+commandResponse.getSender()+"]");
+      debugInfo = "[From Manager : "+commandResponse.getSender()+"]" + debugInfo;
+      LogWrapper.getInstance().info(debugInfo);
+    }
+    commandResult = ResultBuilder.fromJson((String) response);
+
+    //3. Post Remote Execution
+    if (interceptor != null) {
+      Result postExecResult = interceptor.postExecution(parseResult, commandResult);
+      if (postExecResult != null) {
+        if (Status.ERROR.equals(postExecResult.getStatus())) {
+          if (logWrapper.infoEnabled()) {
+            logWrapper.info("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
           }
+        } else if (logWrapper.fineEnabled()) {
+          logWrapper.fine("Post execution Result :: "+ResultBuilder.resultAsString(postExecResult));
         }
-      }// not null response
-    } else {
-      shell.logWarning("Can't execute a remote command without connection. Use 'connect' first to connect.", null);
-      logWrapper.info("Can't execute a remote command \""+parseResult.getUserInput()+"\" without connection. Use 'connect' first to connect to GemFire.");
+        commandResult = postExecResult;
+      }
     }
+
     return commandResult;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
index ab49270..580b6c0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
@@ -46,7 +46,7 @@ public class ResourceOperationContext extends OperationContext {
     //for DATA resource, when we construct the lock to guard the operations, there should always be a 3rd part (regionName),
     // if no regionName is specified, we need to add "NULL" to it.
     // this means, for general data operations, or operations that we can't put a regionName on yet, like backup diskstore, query data, create regions
-    // it will require DATA:REAT/WRITE:NULL role
+    // it will require DATA:READ/WRITE:NULL role
     if(this.resource==Resource.DATA && this.regionName==null){
       this.regionName = "NULL";
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
index c411972..1f6c52a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/AbstractCommandsController.java
@@ -38,6 +38,7 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.internal.util.ArrayUtils;
 import com.gemstone.gemfire.management.DistributedSystemMXBean;
 import com.gemstone.gemfire.management.ManagementService;
@@ -48,9 +49,8 @@ import com.gemstone.gemfire.management.internal.SystemManagementService;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.management.internal.web.controllers.support.LoginHandlerInterceptor;
-import com.gemstone.gemfire.management.internal.web.controllers.support.MemberMXBeanAdapter;
 import com.gemstone.gemfire.management.internal.web.util.UriUtils;
-import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.apache.logging.log4j.Logger;
 import org.springframework.beans.propertyeditors.StringArrayPropertyEditor;
@@ -425,7 +425,6 @@ public abstract class AbstractCommandsController {
    * 
    * @return a proxy instance to the MemberMXBean of the GemFire Manager.
    * @see #getMBeanServer()
-   * @see #createMemberMXBeanForManagerUsingAdapter(javax.management.MBeanServer, javax.management.ObjectName)
    * @see #createMemberMXBeanForManagerUsingProxy(javax.management.MBeanServer, javax.management.ObjectName)
    * @see com.gemstone.gemfire.management.DistributedSystemMXBean
    * @see com.gemstone.gemfire.management.MemberMXBean
@@ -456,21 +455,6 @@ public abstract class AbstractCommandsController {
   }
 
   /**
-   * Creates an Adapter using the Platform MBeanServer and ObjectName to invoke operations on the GemFire Manager's
-   * MemberMXBean.
-   * 
-   * @param server a reference to this JVM's Platform MBeanServer.
-   * @param managingMemberObjectName the ObjectName of the GemFire Manager's MemberMXBean registered in
-   * the Platform MBeanServer.
-   * @return an Adapter for invoking operations on the GemFire Manager's MemberMXBean.
-   * @see com.gemstone.gemfire.management.internal.web.controllers.AbstractCommandsController.MemberMXBeanProxy
-   * @see #createMemberMXBeanForManagerUsingProxy(javax.management.MBeanServer, javax.management.ObjectName)
-   */
-  private MemberMXBean createMemberMXBeanForManagerUsingAdapter(final MBeanServer server, final ObjectName managingMemberObjectName) {
-    return new MemberMXBeanProxy(server, managingMemberObjectName);
-  }
-
-  /**
    * Creates a Proxy using the Platform MBeanServer and ObjectName in order to access attributes and invoke operations
    * on the GemFire Manager's MemberMXBean.
    * 
@@ -478,7 +462,6 @@ public abstract class AbstractCommandsController {
    * @param managingMemberObjectName the ObjectName of the GemFire Manager's MemberMXBean registered in
    * the Platform MBeanServer.
    * @return a Proxy for accessing attributes and invoking operations on the GemFire Manager's MemberMXBean.
-   * @see #createMemberMXBeanForManagerUsingAdapter(javax.management.MBeanServer, javax.management.ObjectName)
    * @see javax.management.JMX#newMXBeanProxy(javax.management.MBeanServerConnection, javax.management.ObjectName, Class)
    */
   private MemberMXBean createMemberMXBeanForManagerUsingProxy(final MBeanServer server, final ObjectName managingMemberObjectName) {
@@ -538,7 +521,7 @@ public abstract class AbstractCommandsController {
   /**
    * Executes the specified command as entered by the user using the GemFire Shell (Gfsh).  Note, Gfsh performs
    * validation of the command during parsing before sending the command to the Manager for processing.
-   * 
+   *
    * @param command a String value containing a valid command String as would be entered by the user in Gfsh.
    * @return a result of the command execution as a String, typically marshalled in JSON to be serialized back to Gfsh.
    * @see com.gemstone.gemfire.management.internal.cli.shell.Gfsh
@@ -549,14 +532,26 @@ public abstract class AbstractCommandsController {
   protected String processCommand(final String command) {
     return processCommand(command, getEnvironment(), null);
   }
+
   protected Callable<ResponseEntity<String>> getProcessCommandCallable(final String command){
     return getProcessCommandCallable(command, null);
   }
 
   protected Callable<ResponseEntity<String>> getProcessCommandCallable(final String command, final byte[][] fileData){
     Callable callable = new Callable<ResponseEntity<String>>() {
-      @Override public ResponseEntity<String> call() throws Exception {
-        return new ResponseEntity<String>(processCommand(command, fileData), HttpStatus.OK);
+      @Override
+      public ResponseEntity<String> call() throws Exception {
+        String result = null;
+        try {
+          result = processCommand(command, fileData);
+        }
+        catch(NotAuthorizedException ex){
+          return new ResponseEntity<String>(ex.getMessage(), HttpStatus.FORBIDDEN);
+        }
+        catch(Exception ex){
+          return new ResponseEntity<String>(ex.getMessage(), HttpStatus.INTERNAL_SERVER_ERROR);
+        }
+        return new ResponseEntity<String>(result, HttpStatus.OK);
       }
     };
     return GeodeSecurityUtil.associateWith(callable);
@@ -614,52 +609,13 @@ public abstract class AbstractCommandsController {
    */
   protected String processCommand(final String command, final Map<String, String> environment, final byte[][] fileData) {
     logger.info(LogMarker.CONFIG, "Processing Command ({}) with Environment ({}) having File Data ({})...", command,
-        environment, (fileData != null));
-    String result =  getManagingMemberMXBean().processCommand(command, environment, ArrayUtils.toByteArray(fileData));
-
-    return result;
+      environment, (fileData != null));
+    return getManagingMemberMXBean().processCommand(command, environment, ArrayUtils.toByteArray(fileData));
   }
 
-
-  /**
-   * The MemberMXBeanProxy class is a proxy for the MemberMXBean interface transforming an operation on the member
-   * MBean into a invocation on the MBeanServer, invoke method.
-   * 
-   * @see com.gemstone.gemfire.management.internal.web.controllers.support.MemberMXBeanAdapter
-   */
-  private static class MemberMXBeanProxy extends MemberMXBeanAdapter {
-
-    private final MBeanServer server;
-
-    private final ObjectName objectName;
-
-    public MemberMXBeanProxy(final MBeanServer server, final ObjectName objectName) {
-      assertNotNull(server, "The connection or reference to the Platform MBeanServer cannot be null!");
-      assertNotNull(objectName, "The JMX ObjectName for the GemFire Manager MemberMXBean cannot be null!");
-      this.server = server;
-      this.objectName = objectName;
-    }
-
-    protected MBeanServer getMBeanServer() {
-      return server;
-    }
-
-    protected ObjectName getObjectName() {
-      return objectName;
-    }
-
-    @Override
-    public String processCommand(final String commandString, final Map<String, String> env) {
-      try {
-        return String.valueOf(getMBeanServer().invoke(getObjectName(), "processCommand",
-          new Object[] { commandString, env }, new String[] { String.class.getName(), Map.class.getName() }));
-      }
-      catch (Exception e) {
-        throw new RuntimeException(String.format(
-          "An error occurred while executing processCommand with command String (%1$s) on the MemberMXBean (%2$s) of the GemFire Manager using environment (%3$s)!",
-            commandString, getObjectName(), env), e);
-      }
-    }
+  @ExceptionHandler(NotAuthorizedException.class)
+  public ResponseEntity<String> handleAppException(NotAuthorizedException ex) {
+    return new ResponseEntity<String>(ex.getMessage(), HttpStatus.FORBIDDEN);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
index b2159d2..944644f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/shell/AbstractHttpOperationInvoker.java
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.management.DistributedSystemMXBean;
 import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
 import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
-import com.gemstone.gemfire.management.internal.security.ResourceConstants;
 import com.gemstone.gemfire.management.internal.web.domain.Link;
 import com.gemstone.gemfire.management.internal.web.domain.QueryParameterSource;
 import com.gemstone.gemfire.management.internal.web.http.ClientHttpRequest;
@@ -51,6 +50,7 @@ import com.gemstone.gemfire.management.internal.web.http.HttpMethod;
 import com.gemstone.gemfire.management.internal.web.http.converter.SerializableObjectHttpMessageConverter;
 import com.gemstone.gemfire.management.internal.web.shell.support.HttpMBeanProxyFactory;
 import com.gemstone.gemfire.management.internal.web.util.UriUtils;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 
 import org.apache.logging.log4j.Logger;
 import org.springframework.http.HttpStatus;
@@ -206,15 +206,17 @@ public abstract class AbstractHttpOperationInvoker implements HttpOperationInvok
 
       @Override
       public void handleError(final ClientHttpResponse response) throws IOException {
-        final String message = String.format("The HTTP request failed with: %1$d - %2$s", response.getRawStatusCode(),
-          response.getStatusText());
-
-        //gfsh.logSevere(message, null);
+        String body = readBody(response);
+        final String message = String.format("The HTTP request failed with: %1$d - %2$s.", response.getRawStatusCode(), body);
 
         if (gfsh.getDebug()) {
-          gfsh.logSevere(readBody(response), null);
+          gfsh.logSevere(body, null);
         }
-        throw new RuntimeException(message);
+
+        if(response.getRawStatusCode()==403)
+          throw new NotAuthorizedException(message);
+        else
+          throw new RuntimeException(message);
       }
 
       private String readBody(final ClientHttpResponse response) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index 377ab77..b21302e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -112,9 +112,11 @@ public class GfshCommandsSecurityTest {
 
 
   private void runCommandsWithAndWithout(String permission) throws Exception{
-    List<TestCommand> permitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
-    for(TestCommand clusterRead:permitted) {
-      LogService.getLogger().info("Processing authorized command: "+clusterRead.getCommand());gfsh.executeCommand(clusterRead.getCommand());
+    List<TestCommand> allPermitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
+    for(TestCommand permitted:allPermitted) {
+      LogService.getLogger().info("Processing authorized command: "+permitted.getCommand());
+
+      gfsh.executeCommand(permitted.getCommand());
       CommandResult result = (CommandResult) gfsh.getResult();
       assertNotNull(result);
 
@@ -127,7 +129,7 @@ public class GfshCommandsSecurityTest {
     }
 
     List<TestCommand> others = TestCommand.getCommands();
-    others.removeAll(permitted);
+    others.removeAll(allPermitted);
     for(TestCommand other:others) {
       // skip no permission commands
       if(other.getPermission()==null)
@@ -135,6 +137,7 @@ public class GfshCommandsSecurityTest {
 
       LogService.getLogger().info("Processing unauthorized command: "+other.getCommand());
       gfsh.executeCommand(other.getCommand());
+
       CommandResult result = (CommandResult) gfsh.getResult();
       int errorCode = ((ErrorResultData) result.getResultData()).getErrorCode();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
index 8261d09..cabf555 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/MemberMBeanSecurityJUnitTest.java
@@ -103,7 +103,7 @@ public class MemberMBeanSecurityJUnitTest {
     assertThatThrownBy(() -> bean.isCacheServer()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.isServer()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.listConnectedGatewayReceivers()).hasMessageContaining(TestCommand.clusterRead.toString());
-    //assertThatThrownBy(() -> bean.processCommand("create region --name=Region_A")).hasMessageContaining("DATA:MANAGE");
+    assertThatThrownBy(() -> bean.processCommand("create region --name=Region_A")).hasMessageContaining("DATA:MANAGE");
     assertThatThrownBy(() -> bean.showJVMMetrics()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.status()).hasMessageContaining(TestCommand.clusterRead.toString());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14a548ff/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
index 03d39fd..2e1aa65 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/web/controllers/WanCommandsControllerJUnitTest.java
@@ -20,14 +20,15 @@ import static com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.*;
 import static junitparams.JUnitParamsRunner.*;
 import static org.assertj.core.api.Assertions.*;
 
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
 
 /**
  * Unit tests for WanCommandsController. 



[27/50] [abbrv] incubator-geode git commit: GEODE-17: format changes after a review

Posted by kl...@apache.org.
GEODE-17: format changes after a review


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

Branch: refs/heads/feature/GEODE-835
Commit: fcd5491a70e54d27f1fa6d57a95c91d2a5e51319
Parents: ec1415f
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed May 18 07:43:06 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed May 18 07:43:06 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/distributed/internal/DistributionConfig.java | 3 +--
 .../gemfire/internal/security/shiro/GeodePermissionResolver.java  | 3 ++-
 .../internal/security/CacheServerMBeanShiroJUnitTest.java         | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcd5491a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
index 36ef671..2dbb5c9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
@@ -52,8 +52,7 @@ import com.gemstone.gemfire.memcached.GemFireMemcachedServer;
  *
  * @since 2.1
  */
-public interface
-DistributionConfig extends Config, LogConfig {
+public interface DistributionConfig extends Config, LogConfig {
 
   ////////////////////  Instance Methods  ////////////////////
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcd5491a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
index d170756..44f6685 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
@@ -22,7 +22,8 @@ import org.apache.shiro.authz.permission.PermissionResolver;
 import org.apache.shiro.authz.permission.WildcardPermission;
 
 public class GeodePermissionResolver implements PermissionResolver {
-  @Override public Permission resolvePermission(final String permissionString) {
+  @Override
+  public Permission resolvePermission(final String permissionString) {
     return new WildcardPermission(permissionString, true);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcd5491a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
index 1c8586f..85a55a7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
@@ -83,7 +83,7 @@ public class CacheServerMBeanShiroJUnitTest {
 
   @Test
   @JMXConnectionConfiguration(user = "dataReader", password = "12345")
-  public void ztestDataRead() throws Exception{
+  public void testDataRead() throws Exception{
     assertThatThrownBy(() -> bean.removeIndex("foo")).hasMessageContaining(TestCommand.dataManage.toString());
     assertThatThrownBy(() -> bean.fetchLoadProbe()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.getActiveCQCount()).hasMessageContaining(TestCommand.clusterRead.toString());


[50/50] [abbrv] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-835

Posted by kl...@apache.org.
Merge remote-tracking branch 'origin/develop' into feature/GEODE-835


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

Branch: refs/heads/feature/GEODE-835
Commit: 7e559224ad24789f288f1ef02c3b6029ceee5547
Parents: 29861e4 8a3c351
Author: Kirk Lund <kl...@pivotal.io>
Authored: Fri May 20 09:59:16 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Fri May 20 09:59:16 2016 -0700

----------------------------------------------------------------------
 KEYS                                            |   37 +
 extensions/geode-modules-assembly/build.gradle  |   53 +-
 .../release/conf/cache-client.xml               |   14 +-
 .../release/conf/cache-server.xml               |    6 +-
 .../release/scripts/cacheserver.bat             |  133 -
 .../release/scripts/cacheserver.sh              |   97 -
 .../release/scripts/gemfire.bat                 |   41 -
 .../release/scripts/gemfire.sh                  |   58 -
 .../release/scripts/setenv.properties           |    1 +
 .../release/session/bin/cacheserver.bat         |   56 -
 .../release/session/bin/cacheserver.sh          |   57 -
 .../release/session/bin/gemfire.bat             |   53 -
 .../release/session/bin/gemfire.sh              |   69 -
 .../release/session/bin/modify_war              |   91 +-
 .../release/session/bin/setenv.properties       |    6 -
 .../gemfire-cs-tomcat-7/context-fragment.xml    |   15 -
 .../tcserver/gemfire-cs-tomcat-7/modules.env    |    1 -
 .../gemfire-cs-tomcat-8/context-fragment.xml    |   15 -
 .../tcserver/gemfire-cs-tomcat-8/modules.env    |    1 -
 .../gemfire-cs/configuration-prompts.properties |   17 -
 .../tcserver/gemfire-cs/context-fragment.xml    |   13 -
 .../release/tcserver/gemfire-cs/modules.env     |    1 -
 .../tcserver/gemfire-cs/server-fragment.xml     |   12 -
 .../gemfire-p2p-tomcat-7/context-fragment.xml   |   15 -
 .../tcserver/gemfire-p2p-tomcat-7/modules.env   |    1 -
 .../gemfire-p2p-tomcat-8/context-fragment.xml   |   15 -
 .../tcserver/gemfire-p2p-tomcat-8/modules.env   |    1 -
 .../configuration-prompts.properties            |   19 -
 .../tcserver/gemfire-p2p/context-fragment.xml   |   13 -
 .../release/tcserver/gemfire-p2p/modules.env    |    1 -
 .../tcserver/gemfire-p2p/server-fragment.xml    |   14 -
 .../geode-cs-tomcat-7/context-fragment.xml      |   15 +
 .../geode-cs-tomcat-8/context-fragment.xml      |   15 +
 .../geode-cs/configuration-prompts.properties   |   17 +
 .../tcserver/geode-cs/context-fragment.xml      |   13 +
 .../tcserver/geode-cs/server-fragment.xml       |   12 +
 .../geode-p2p-tomcat-7/context-fragment.xml     |   15 +
 .../geode-p2p-tomcat-8/context-fragment.xml     |   15 +
 .../geode-p2p/configuration-prompts.properties  |   19 +
 .../tcserver/geode-p2p/context-fragment.xml     |   13 +
 .../tcserver/geode-p2p/server-fragment.xml      |   14 +
 extensions/geode-modules-session/build.gradle   |    2 +-
 .../junit/SeparateClassloaderTestRunner.java    |   56 -
 extensions/geode-modules-tomcat7/build.gradle   |    6 +
 .../gemstone/gemfire/modules/util/Banner.java   |   20 +-
 geode-assembly/build.gradle                     |   10 +-
 .../LocatorLauncherAssemblyIntegrationTest.java |    5 +
 .../SharedConfigurationEndToEndDUnitTest.java   |   13 +-
 .../src/test/resources/expected_jars.txt        |    1 +
 geode-core/build.gradle                         |    2 +-
 .../gemfire/cache/AttributesFactory.java        |    1 +
 .../gemfire/cache/AttributesMutator.java        |    1 +
 .../gemstone/gemfire/cache/RegionFactory.java   |    1 +
 .../cache/asyncqueue/AsyncEventQueue.java       |    9 +
 .../asyncqueue/AsyncEventQueueFactory.java      |   13 +
 .../internal/AsyncEventQueueFactoryImpl.java    |    8 +-
 .../internal/AsyncEventQueueImpl.java           |    9 +-
 .../cache/client/ClientRegionFactory.java       |    1 +
 .../client/internal/ClientMetadataService.java  |   31 +-
 .../cache/operations/OperationContext.java      |   10 +-
 .../query/internal/index/AbstractIndex.java     |   12 +-
 .../cache/util/BoundedLinkedHashMap.java        |   88 -
 .../gemfire/cache/wan/GatewaySender.java        |    2 +
 .../gemstone/gemfire/distributed/Locator.java   |    9 +-
 .../gemfire/distributed/LocatorLauncher.java    |    4 +
 .../internal/AbstractDistributionConfig.java    |    2 +-
 .../internal/DistributionConfig.java            |   23 +-
 .../distributed/internal/InternalLocator.java   |   50 +-
 .../distributed/internal/LocatorStats.java      |   31 -
 .../distributed/internal/ServerLocator.java     |   44 +-
 .../internal/SharedConfiguration.java           |    2 +-
 .../membership/gms/membership/GMSJoinLeave.java |   23 +-
 .../gms/messenger/JGroupsMessenger.java         |   10 +-
 .../internal/tcpserver/TcpServer.java           |   41 +-
 .../gemfire/internal/AbstractConfig.java        |    2 +-
 .../InsufficientDiskSpaceException.java         |   53 -
 .../gemfire/internal/cache/AbstractRegion.java  |    2 +-
 .../internal/cache/AbstractRegionEntry.java     |   14 +-
 .../internal/cache/AbstractRegionMap.java       |   26 +-
 .../gemfire/internal/cache/BucketRegion.java    |   10 +
 .../internal/cache/BucketRegionQueue.java       |   10 +-
 .../gemfire/internal/cache/EntryEventImpl.java  |    2 +-
 .../gemfire/internal/cache/LocalRegion.java     |   52 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |    7 +-
 .../gemfire/internal/cache/RegionEntry.java     |    2 -
 .../internal/cache/RemoteDestroyMessage.java    |    2 +-
 .../PartitionedRegionRebalanceOp.java           |  322 +-
 .../rebalance/BucketOperatorImpl.java           |   78 +
 .../rebalance/BucketOperatorWrapper.java        |  235 ++
 .../rebalance/PartitionedRegionLoadModel.java   |    2 +
 .../internal/cache/versions/VersionTag.java     |   83 +-
 .../cache/wan/AbstractGatewaySender.java        |  100 +-
 .../cache/wan/GatewaySenderAttributes.java      |    7 +
 .../parallel/ParallelQueueRemovalMessage.java   |    1 +
 .../cache/xmlcache/AsyncEventQueueCreation.java |   11 +
 .../internal/cache/xmlcache/CacheXml.java       |    1 +
 .../cache/xmlcache/CacheXmlGenerator.java       |    8 +
 .../internal/cache/xmlcache/CacheXmlParser.java |    7 +
 .../gemfire/internal/logging/LogService.java    |    2 +-
 .../internal/logging/log4j/LogWriterLogger.java |    2 +-
 .../message/GemFireParameterizedMessage.java    |  556 ++++
 .../GemFireParameterizedMessageFactory.java     |   56 +
 .../internal/offheap/FreeListManager.java       |  292 +-
 .../offheap/OffHeapRegionEntryHelper.java       |    4 +-
 .../OffHeapStoredObjectAddressStack.java        |    5 +-
 .../gemfire/internal/offheap/SlabImpl.java      |   11 +-
 .../internal/process/ControllableProcess.java   |   10 +-
 .../internal/security/GeodeSecurityUtil.java    |  123 +-
 .../security/shiro/GeodePermissionResolver.java |   29 +
 .../security/shiro/JMXShiroAuthenticator.java   |   13 +-
 .../internal/security/shiro/ShiroPrincipal.java |   39 +
 .../gemfire/internal/util/BlobHelper.java       |   75 +-
 .../management/internal/ManagementAgent.java    |    7 +-
 .../internal/SystemManagementService.java       |   29 +-
 .../internal/beans/MemberMBeanBridge.java       |   18 +-
 .../CreateAlterDestroyRegionCommands.java       |    5 +-
 .../internal/cli/commands/DataCommands.java     |    1 -
 .../internal/cli/commands/IndexCommands.java    |   32 +-
 .../cli/commands/LauncherLifecycleCommands.java |    2 +-
 .../internal/cli/commands/QueueCommands.java    |   18 +-
 .../internal/cli/commands/ShellCommands.java    |    4 +-
 .../functions/AsyncEventQueueFunctionArgs.java  |  134 +
 .../CreateAsyncEventQueueFunction.java          |   54 +-
 .../cli/functions/DataCommandFunction.java      |   11 +-
 .../internal/cli/i18n/CliStrings.java           |    4 +-
 .../internal/cli/remote/CommandProcessor.java   |    9 +-
 .../internal/cli/result/ResultBuilder.java      |    3 +-
 .../cli/shell/GfshExecutionStrategy.java        |  230 +-
 .../security/ResourceOperationContext.java      |   10 +-
 .../controllers/AbstractCommandsController.java |   88 +-
 .../controllers/QueueCommandsController.java    |    2 +
 .../support/MemberMXBeanAdapter.java            |  654 ----
 .../web/shell/AbstractHttpOperationInvoker.java |   16 +-
 .../com/gemstone/gemfire/pdx/JSONFormatter.java |  103 +-
 .../pdx/internal/json/PdxListHelper.java        |    2 +-
 .../message/GemFireParameterizedMessage.java    |  554 ----
 .../GemFireParameterizedMessageFactory.java     |   54 -
 .../geode.apache.org/schema/cache/cache-1.0.xsd |    1 +
 .../membership/gms/messenger/jgroups-mcast.xml  |  194 +-
 .../com/gemstone/gemfire/DeltaTestImpl.java     |   10 +
 ...ventQueueEvictionAndExpirationJUnitTest.java |  346 ++
 .../cache30/CacheXmlGeode10DUnitTest.java       |   78 +
 ...tedAckOverflowRegionCCEOffHeapDUnitTest.java |    2 +-
 ...dAckPersistentRegionCCEOffHeapDUnitTest.java |    2 +-
 ...DistributedAckRegionCCEOffHeapDUnitTest.java |    2 +-
 .../DistributedAckRegionOffHeapDUnitTest.java   |    2 +-
 ...stributedNoAckRegionCCEOffHeapDUnitTest.java |    2 +-
 .../DistributedNoAckRegionOffHeapDUnitTest.java |    2 +-
 .../GlobalRegionCCEOffHeapDUnitTest.java        |    2 +-
 .../cache30/GlobalRegionOffHeapDUnitTest.java   |    2 +-
 .../OffHeapLRUEvictionControllerDUnitTest.java  |    2 +-
 .../PartitionedRegionOffHeapDUnitTest.java      |    2 +-
 .../gemfire/cache30/ReconnectDUnitTest.java     |   57 +-
 ...tractLocatorLauncherIntegrationTestCase.java |   21 +-
 .../DistributedLockServiceDUnitTest.java        |   46 +-
 .../distributed/HostedLocatorsDUnitTest.java    |  110 +
 .../gemfire/distributed/LocatorJUnitTest.java   |   66 +-
 ...LocatorLauncherLocalFileIntegrationTest.java |   12 +-
 .../LocatorLauncherLocalIntegrationTest.java    |   26 +-
 ...ocatorLauncherRemoteFileIntegrationTest.java |    7 +-
 .../LocatorLauncherRemoteIntegrationTest.java   |   31 +
 ...rRemoteWithCustomLoggingIntegrationTest.java |    5 +
 .../InternalDistributedSystemJUnitTest.java     |   30 +-
 .../internal/ServerLocatorJUnitTest.java        |    0
 .../gms/membership/GMSJoinLeaveJUnitTest.java   |    2 +-
 ...DistrbutedRegionProfileOffHeapDUnitTest.java |    2 +-
 .../cache/OffHeapEvictionDUnitTest.java         |    2 +-
 .../cache/OffHeapEvictionStatsDUnitTest.java    |    2 +-
 ...nedRegionLocalMaxMemoryOffHeapDUnitTest.java |    2 +-
 ...rtitionedRegionOffHeapEvictionDUnitTest.java |    2 +-
 .../PartitionedRegionSingleHopDUnitTest.java    |  449 +--
 .../internal/cache/SingleHopStatsDUnitTest.java |  448 ++-
 .../control/RebalanceOperationDUnitTest.java    |  148 +-
 ...tentColocatedPartitionedRegionDUnitTest.java |    3 +-
 .../rebalance/BucketOperatorImplTest.java       |  138 +
 .../rebalance/BucketOperatorWrapperTest.java    |  323 ++
 ...ForceInvalidateOffHeapEvictionDUnitTest.java |    2 +-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |   38 +-
 .../sockets/UpdatePropagationDUnitTest.java     |  489 +--
 .../sockets/UpdatePropagationPRDUnitTest.java   |    4 +-
 .../versions/AbstractVersionTagTestBase.java    |   92 +
 .../cache/versions/VMVersionTagTest.java        |   32 +
 ...ompressionCacheListenerOffHeapDUnitTest.java |    2 +-
 ...ressionRegionOperationsOffHeapDUnitTest.java |    2 +-
 .../internal/offheap/FreeListManagerTest.java   |   96 +-
 .../OffHeapRegionEntryHelperJUnitTest.java      |    4 +-
 .../offheap/OutOfOffHeapMemoryDUnitTest.java    |    2 +-
 .../gemfire/internal/util/BlobHelperTest.java   |  179 ++
 ...bHelperWithThreadContextClassLoaderTest.java |  306 ++
 .../gemfire/internal/util/SerializableImpl.java |   37 -
 .../util/SerializableImplWithValue.java         |   56 -
 .../gemfire/internal/util/Valuable.java         |    1 -
 .../management/LocatorManagementDUnitTest.java  |   63 +-
 .../cli/commands/CliCommandTestBase.java        |   51 +-
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  |  305 --
 .../commands/HelpCommandsIntegrationTest.java   |  140 +
 .../cli/commands/QueueCommandsDUnitTest.java    |    2 +
 .../SharedConfigurationCommandsDUnitTest.java   |    8 +-
 .../SharedConfigurationDUnitTest.java           |    2 +-
 .../SharedConfigurationTestUtils.java           |    2 +-
 .../SharedConfigurationUsingDirDUnitTest.java   |    3 +-
 .../security/CliCommandsSecurityTest.java       |   24 +-
 .../security/DataCommandsSecurityTest.java      |    4 +-
 .../GeodeSecurityUtilCustomRealmJUnitTest.java  |   18 +-
 .../GeodeSecurityUtilWithIniFileJUnitTest.java  |   15 +-
 .../security/GfshCommandsSecurityTest.java      |   42 +-
 .../security/MemberMBeanSecurityJUnitTest.java  |    2 +-
 .../ResourceOperationContextJUnitTest.java      |   11 +-
 .../internal/security/ShiroCacheStartRule.java  |    2 +-
 .../internal/security/TestCommand.java          |   23 +-
 .../WanCommandsControllerJUnitTest.java         |    7 +-
 .../gemfire/pdx/JSONFormatterJUnitTest.java     |  208 ++
 .../pdx/JSONPdxClientServerDUnitTest.java       |   10 +-
 .../pdx/PdxFormatterPutGetJUnitTest.java        |  208 --
 .../gemfire/pdx/TestObjectForJSONFormatter.java | 1000 ++++++
 .../gemfire/pdx/TestObjectForPdxFormatter.java  | 1003 ------
 .../test/dunit/standalone/DUnitLauncher.java    |   19 +-
 .../test/dunit/standalone/ProcessManager.java   |   45 +-
 .../com/gemstone/gemfire/test/fake/Fakes.java   |   19 +
 .../gemfire/test/golden/log4j2-test.xml         |   18 -
 .../gemfire/codeAnalysis/excludedClasses.txt    |    2 +
 .../sanctionedDataSerializables.txt             |    4 +-
 .../codeAnalysis/sanctionedSerializables.txt    |    6 +-
 .../cli/commands/golden-help-offline.properties | 3003 ++++++++++++++++++
 .../cli/commands/golden-help-online.properties  |  657 ++++
 .../internal/security/cacheServer.json          |   42 +
 .../pdx/jsonStrings/jsonListInsideList.txt      |   48 +
 .../src/test/resources/ssl/trusted.keystore     |  Bin 1078 -> 2241 bytes
 .../cache/PRDeltaPropagationDUnitTest.java      |  173 +-
 .../cache/ha/HADispatcherDUnitTest.java         |  608 ++--
 .../gemfire/test/junit/rules/DiskDirRule.java   |   54 +
 .../test/junit/runner/SuiteBlockRunner.java     |   46 +
 .../gemfire/test/junit/runner/SuiteRunner.java  |   53 +
 .../CategoryWithParameterizedRunner.java        |   44 +
 .../CategoryWithParameterizedRunnerFactory.java |   42 +
 .../junit/runners/ExposedGetAnnotations.java    |   23 +
 .../test/junit/rules/DiskDirRuleTest.java       |   49 +
 ...egoryWithParameterizedRunnerFactoryTest.java |  162 +
 .../gemfire/cache/lucene/LuceneIndex.java       |    7 +
 .../gemfire/cache/lucene/LuceneService.java     |    4 +-
 .../lucene/internal/LuceneEventListener.java    |    2 +
 .../LuceneIndexForPartitionedRegion.java        |   70 +-
 .../LuceneIndexForReplicatedRegion.java         |    1 +
 .../cache/lucene/internal/LuceneIndexImpl.java  |   59 +-
 .../lucene/internal/LuceneQueryFactoryImpl.java |    3 +
 .../internal/PartitionedRepositoryManager.java  |    1 +
 .../internal/xml/LuceneIndexCreation.java       |    5 +
 .../LuceneIndexCreationIntegrationTest.java     |  192 ++
 ...ceneIndexCreationOffHeapIntegrationTest.java |   72 +
 ...IndexCreationPersistenceIntegrationTest.java |  136 +
 .../cache/lucene/LuceneIntegrationTest.java     |   63 +
 .../gemfire/cache/lucene/LuceneQueriesBase.java |   44 +-
 .../lucene/LuceneQueriesIntegrationTest.java    |   61 +-
 .../cache/lucene/LuceneQueriesPRBase.java       |    2 +
 .../lucene/LuceneQueriesPeerPRDUnitTest.java    |    1 +
 .../LuceneQueriesPeerPROverflowDUnitTest.java   |    2 +
 .../LuceneIndexForPartitionedRegionTest.java    |  138 +-
 .../internal/LuceneIndexImplJUnitTest.java      |   78 +
 .../LuceneIndexRecoveryHAIntegrationTest.java   |   14 +-
 .../LuceneQueryFactoryImplJUnitTest.java        |   21 +-
 .../LuceneQueryImplIntegrationTest.java         |   14 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |   96 +
 .../gemfire/cache/lucene/test/TestObject.java   |   52 +
 .../gemfire/cache/lucene/test/package-info.java |   23 +
 .../cache/wan/Simple2CacheServerDUnitTest.java  |   47 +-
 .../ClusterConfigurationDUnitTest.java          |   10 +-
 geode-web/build.gradle                          |   11 +
 .../cli/commands/CommandOverHttpDUnitTest.java  |   58 +
 .../ConnectCommandWithHttpAndSSLDUnitTest.java  |  305 ++
 .../GfshCommandsOverHttpSecurityTest.java       |   29 +
 gradle/dependency-versions.properties           |    1 +
 gradle/ide.gradle                               |   15 +-
 gradle/rat.gradle                               |    1 +
 gradle/test.gradle                              |    8 +
 274 files changed, 12764 insertions(+), 6987 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7e559224/geode-assembly/build.gradle
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7e559224/geode-core/build.gradle
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7e559224/gradle/dependency-versions.properties
----------------------------------------------------------------------


[26/50] [abbrv] incubator-geode git commit: GEODE-1392: add tests for BlobHelper

Posted by kl...@apache.org.
GEODE-1392: add tests for BlobHelper

* create BlobHelperTest with basic unit tests
* create BlobHelperWithThreadContextClassLoaderTest with TCCL related tests
* remove deadcode
* minor cleanup of format


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

Branch: refs/heads/feature/GEODE-835
Commit: ec1415f4ef6268c524a287a8a9cc19cd0fd9679c
Parents: 46056a6
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue May 17 09:38:43 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue May 17 17:26:10 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/util/BlobHelper.java       |  75 ++---
 .../gemfire/internal/util/BlobHelperTest.java   | 179 +++++++++++
 ...bHelperWithThreadContextClassLoaderTest.java | 306 +++++++++++++++++++
 .../gemfire/internal/util/SerializableImpl.java |  37 ---
 .../util/SerializableImplWithValue.java         |  56 ----
 .../gemfire/internal/util/Valuable.java         |   1 -
 6 files changed, 509 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec1415f4/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
index 28252c3..45f9774 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/BlobHelper.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.internal.util;
 
 import java.io.IOException;
@@ -26,19 +25,17 @@ import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.DSCODE;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.pdx.internal.PdxInputStream;
 
 /**
  * A "blob" is a serialized representation of an object into a byte[].
- * BlobHelper provides utility methods for
- * serializing and deserializing the object.
- * 
- * 
+ * BlobHelper provides utility methods for serializing and deserializing the
+ * object.
+ *
+ * TODO: compare performance with org.apache.commons.lang.SerializationUtils
  */
-
 public class BlobHelper {
 
   /**
@@ -50,12 +47,10 @@ public class BlobHelper {
   }
 
   /**
-   * A blob is a serialized Object.  This method serializes the
-   * object into a blob and returns the byte array that contains the blob.
+   * A blob is a serialized Object.  This method serializes the object into a
+   * blob and returns the byte array that contains the blob.
    */
-  public static byte[] serializeToBlob(Object obj, Version version)
-  throws IOException
-  {
+  public static byte[] serializeToBlob(Object obj, Version version) throws IOException {
     final long start = startSerialization();
     HeapDataOutputStream hdos = new HeapDataOutputStream(version);
     DataSerializer.writeObject(obj, hdos);
@@ -65,45 +60,32 @@ public class BlobHelper {
   }
 
   /**
-   * A blob is a serialized Object.  This method serializes the
-   * object into the given HeapDataOutputStream.
+   * A blob is a serialized Object.  This method serializes the object into
+   * the given HeapDataOutputStream.
    */
-  public static void serializeTo(Object obj, HeapDataOutputStream hdos)
-    throws IOException
-  {
+  public static void serializeTo(Object obj, HeapDataOutputStream hdos) throws IOException {
     final int startBytes = hdos.size();
     final long start = startSerialization();
     DataSerializer.writeObject(obj, hdos);
     endSerialization(start, hdos.size()-startBytes);
   }
-                                                                        
-
 
   /**
-   * A blob is a serialized Object.  This method 
-   * returns the deserialized object.
+   * A blob is a serialized Object.  This method returns the deserialized
+   * object.
    */
-  public static Object deserializeBlob(byte[] blob) throws IOException,
-      ClassNotFoundException {
+  public static Object deserializeBlob(byte[] blob) throws IOException, ClassNotFoundException {
     return deserializeBlob(blob, null, null);
   }
 
   /**
-   * A blob is a serialized Object.  This method 
-   * returns the deserialized object.
+   * A blob is a serialized Object.  This method returns the deserialized
+   * object.
    */
-  public static Object deserializeBlob(byte[] blob, Version version,
-      ByteArrayDataInput in) throws IOException, ClassNotFoundException {
+  public static Object deserializeBlob(byte[] blob, Version version, ByteArrayDataInput in) throws IOException, ClassNotFoundException {
     Object result;
     final long start = startDeserialization();
-    /*
-    final StaticSystemCallbacks sysCb;
-    if (version != null && (sysCb = GemFireCacheImpl.FactoryStatics
-        .systemCallbacks) != null) {
-      // may need to change serialized shape for SQLFire
-      result = sysCb.fromVersion(blob, true, version, in);
-    }
-    else*/ if (blob.length > 0 && blob[0] == DSCODE.PDX) {
+    if (blob.length > 0 && blob[0] == DSCODE.PDX) {
       // If the first byte of blob indicates a pdx then wrap
       // blob in a PdxInputStream instead.
       // This will prevent us from making a copy of the byte[]
@@ -121,24 +103,13 @@ public class BlobHelper {
       result = DataSerializer.readObject(in);
     }
     endDeserialization(start, blob.length);
-    // this causes a small performance drop in d-no-ack performance tests
-//    if (dis.available() != 0) {
-//      LogWriterI18n lw = InternalDistributedSystem.getLoggerI18n();
-//      if (lw != null && lw.warningEnabled()) {
-//        lw.warning(
-//            LocalizedStrings.BlobHelper_DESERIALIZATION_OF_A_0_DID_NOT_READ_1_BYTES_THIS_INDICATES_A_LOGIC_ERROR_IN_THE_SERIALIZATION_CODE_FOR_THIS_CLASS,
-//            new Object[] {((result!=null) ? result.getClass().getName() : "NULL"), Integer.valueOf(dis.available())});   
-//            
-//      }
-//    }
     return result;
   }
 
   /**
-   * A blob is a serialized Object.  This method 
-   * returns the deserialized object.
-   * If a PdxInstance is returned then it will refer to Chunk's off-heap memory
-   * with an unretained reference.
+   * A blob is a serialized Object.  This method returns the deserialized
+   * object. If a PdxInstance is returned then it will refer to Chunk's
+   * off-heap memory with an unretained reference.
    */
   public static @Unretained Object deserializeOffHeapBlob(StoredObject blob) throws IOException, ClassNotFoundException {
     Object result;
@@ -152,8 +123,10 @@ public class BlobHelper {
     return result;
   }
 
-  public static Object deserializeBuffer(ByteArrayDataInput in, int numBytes)
-      throws IOException, ClassNotFoundException {
+  /**
+   * Unused
+   */
+  public static Object deserializeBuffer(ByteArrayDataInput in, int numBytes) throws IOException, ClassNotFoundException {
     final long start = startDeserialization();
     Object result = DataSerializer.readObject(in);
     endDeserialization(start, numBytes);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec1415f4/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperTest.java
new file mode 100644
index 0000000..8603c31
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.util;
+
+import static com.gemstone.gemfire.internal.util.BlobHelper.*;
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.EOFException;
+import java.io.NotSerializableException;
+import java.io.ObjectInputStream;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.internal.ByteArrayDataInput;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit Tests for {@link BlobHelper}.
+ *
+ * TODO: add coverage for additional methods:
+ * <li>{@link BlobHelper#deserializeBlob(byte[], Version, ByteArrayDataInput)}
+ * <li>{@link BlobHelper#deserializeBuffer(ByteArrayDataInput, int)}
+ * <li>{@link BlobHelper#deserializeOffHeapBlob(StoredObject)}
+ * <li>{@link BlobHelper#serializeToBlob(Object, Version)}
+ */
+@Category(UnitTest.class)
+public class BlobHelperTest {
+
+  private static final int HDOS_ALLOC_SIZE = 32;
+
+  private static final String CLASS_NOT_FOUND_MESSAGE = "ClassNotFoundSerialization.readObject fake exception";
+
+  private Map<Object, Object> mapWithTwoEntries;
+
+  private byte[] bytesOfClassNotFoundSerialization;
+  private byte[] bytesOfMap;
+  private byte[] bytesOfNull;
+  private byte[] zeroBytes;
+
+  @Before
+  public void setUp() throws Exception {
+    this.mapWithTwoEntries = new HashMap<>();
+    this.mapWithTwoEntries.put("FOO", "foo");
+    this.mapWithTwoEntries.put("BAR", 7);
+
+    HeapDataOutputStream hdos = createHeapDataOutputStream();
+    DataSerializer.writeObject(new ClassNotFoundSerialization(), hdos);
+    this.bytesOfClassNotFoundSerialization = hdos.toByteArray();
+
+    hdos = createHeapDataOutputStream();
+    DataSerializer.writeObject(this.mapWithTwoEntries, hdos);
+    this.bytesOfMap = hdos.toByteArray();
+
+    this.bytesOfNull = serializeToBlob(null);
+
+    this.zeroBytes = new byte[0];
+  }
+
+  @Test
+  public void deserializeBlobOfClassNotFoundSerializationThrowsEOFException() throws Exception {
+    assertThatThrownBy(() -> deserializeBlob(this.bytesOfClassNotFoundSerialization)).isExactlyInstanceOf(ClassNotFoundException.class);
+  }
+
+  @Test
+  public void deserializeBlobOfMapReturnsCopyOfMap() throws Exception {
+    final Object object = deserializeBlob(this.bytesOfMap);
+
+    assertThat(object).isNotNull();
+    assertThat(object).isExactlyInstanceOf(HashMap.class);
+    assertThat(object).isNotSameAs(this.mapWithTwoEntries);
+    assertThat(object).isEqualTo(this.mapWithTwoEntries);
+  }
+
+  @Test
+  public void deserializeBlobOfNullReturnsNull() throws Exception {
+    assertThat(deserializeBlob(this.bytesOfNull)).isNull();
+  }
+
+  @Test
+  public void deserializeBlobOfZeroBytesThrowsEOFException() throws Exception {
+    assertThatThrownBy(() -> deserializeBlob(this.zeroBytes)).isExactlyInstanceOf(EOFException.class);
+  }
+
+  @Test
+  public void deserializeBlobWithNullThrowsNullPointerException() throws Exception {
+    assertThatThrownBy(() -> deserializeBlob(null)).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void serializeMapToStreamWritesMapAsBytes() throws Exception {
+    HeapDataOutputStream hdos = createHeapDataOutputStream();
+
+    serializeTo(this.mapWithTwoEntries, hdos);
+
+    assertThat(hdos.toByteArray()).isNotNull().isEqualTo(bytesOfMap);
+  }
+
+  @Test
+  public void serializeNullToStreamWritesNullAsBytes() throws Exception {
+    HeapDataOutputStream hdos = createHeapDataOutputStream();
+
+    serializeTo(null, hdos);
+
+    assertThat(hdos.toByteArray()).isNotNull().isEqualTo(this.bytesOfNull);
+  }
+
+  @Test
+  public void serializeToBlobMapReturnsBytesOfMap() throws Exception {
+    byte[] bytes = serializeToBlob(this.mapWithTwoEntries);
+
+    assertThat(bytes).isNotNull().isEqualTo(this.bytesOfMap);
+  }
+
+  @Test
+  public void serializeToBlobUnserializableThrowsNotSerializableException() throws Exception {
+    assertThatThrownBy(() -> serializeToBlob(new Object()))
+      .isExactlyInstanceOf(NotSerializableException.class)
+      .hasMessage(Object.class.getName());
+  }
+
+  @Test
+  public void serializeToBlobWithNullReturnsBytesOfNull() throws Exception {
+    byte[] bytes = serializeToBlob(null);
+
+    assertThat(bytes).isNotNull().isEqualTo(this.bytesOfNull);
+  }
+
+  @Test
+  public void serializeToNullNullThrowsNullPointerException() throws Exception {
+    assertThatThrownBy(() -> serializeTo(null, null)).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void serializeToNullStreamThrowsNullPointerException() throws Exception {
+    assertThatThrownBy(() -> serializeTo(this.mapWithTwoEntries, null)).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void serializeUnserializableToStreamThrowsNotSerializableException() throws Exception {
+    HeapDataOutputStream hdos = createHeapDataOutputStream();
+
+    assertThatThrownBy(() -> serializeTo(new Object(), hdos))
+      .isExactlyInstanceOf(NotSerializableException.class)
+      .hasMessage(Object.class.getName());
+  }
+
+  private HeapDataOutputStream createHeapDataOutputStream() {
+    return new HeapDataOutputStream(HDOS_ALLOC_SIZE, null, true);
+  }
+
+  private static class ClassNotFoundSerialization implements Serializable {
+    private void readObject(final ObjectInputStream in) throws ClassNotFoundException    {
+      throw new ClassNotFoundException(CLASS_NOT_FOUND_MESSAGE);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec1415f4/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperWithThreadContextClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperWithThreadContextClassLoaderTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperWithThreadContextClassLoaderTest.java
new file mode 100644
index 0000000..6dfcadd
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/BlobHelperWithThreadContextClassLoaderTest.java
@@ -0,0 +1,306 @@
+/*
+ * 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.util;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.bcel.Constants;
+import org.apache.bcel.classfile.Field;
+import org.apache.bcel.classfile.JavaClass;
+import org.apache.bcel.generic.ClassGen;
+import org.apache.bcel.generic.ConstantPoolGen;
+import org.apache.bcel.generic.FieldGen;
+import org.apache.bcel.generic.InstructionFactory;
+import org.apache.bcel.generic.InstructionHandle;
+import org.apache.bcel.generic.InstructionList;
+import org.apache.bcel.generic.MethodGen;
+import org.apache.bcel.generic.Type;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link BlobHelper} with Thread Context ClassLoader.
+ *
+ * @since 2.0.2
+ */
+@Category(UnitTest.class)
+public class BlobHelperWithThreadContextClassLoaderTest {
+
+  private static final String CLASS_NAME_SERIALIZABLE_IMPL = "com.gemstone.gemfire.internal.util.SerializableImpl";
+  private static final String CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE = "com.gemstone.gemfire.internal.util.SerializableImplWithValue";
+  private static final String VALUE = "value";
+  private static final String SET_VALUE = "setValue";
+  private static final String GET_VALUE = "getValue";
+
+  private ClassLoader oldCCL;
+
+  @Before
+  public void setUp() throws MalformedURLException {
+    this.oldCCL = Thread.currentThread().getContextClassLoader();
+    Thread.currentThread().setContextClassLoader(new GeneratingClassLoader(this.oldCCL));
+  }
+
+  @After
+  public void tearDown() {
+    Thread.currentThread().setContextClassLoader(this.oldCCL);
+  }
+
+  @Test
+  public void tcclLoadsSerializableImpl() throws Exception {
+    Class loadedClass = Class.forName(CLASS_NAME_SERIALIZABLE_IMPL, true, Thread.currentThread().getContextClassLoader());
+    assertThat(loadedClass).isNotNull();
+    assertThat(loadedClass.getName()).isEqualTo(CLASS_NAME_SERIALIZABLE_IMPL);
+
+    Object instance = loadedClass.newInstance();
+    assertThat(instance).isNotNull();
+    assertThat(Serializable.class.isInstance(loadedClass));
+    assertThat(loadedClass.getInterfaces()).contains(Serializable.class);
+  }
+
+  @Test
+  public void tcclLoadsSerializableImplWithValue() throws Exception {
+    Class loadedClass = Class.forName(CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE, true, Thread.currentThread().getContextClassLoader());
+    assertThat(loadedClass).isNotNull();
+    assertThat(loadedClass.getName()).isEqualTo(CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE);
+
+    Object instance = loadedClass.newInstance();
+    assertThat(instance).isNotNull();
+
+    assertThat(loadedClass.getSuperclass().getName()).isEqualTo(CLASS_NAME_SERIALIZABLE_IMPL);
+    assertThat(Serializable.class.isInstance(loadedClass));
+
+    assertThat(Valuable.class.isInstance(loadedClass));
+    assertThat(loadedClass.getInterfaces()).contains(Valuable.class);
+
+    Method setter = loadedClass.getMethod("setValue", Object.class);
+    assertThat(setter).isNotNull();
+  }
+
+  /**
+   * Tests serializing an object loaded with the current context class
+   * loader (whose parent is the loader that loads GemFire and test
+   * classes).
+   */
+  @Test
+  public void handlesClassFromOtherClassLoader() throws Exception {
+    Class loadedClass = Class.forName(CLASS_NAME_SERIALIZABLE_IMPL, true, Thread.currentThread().getContextClassLoader());
+
+    Object instance = loadedClass.newInstance();
+    byte[] bytes = BlobHelper.serializeToBlob(instance);
+
+    Object object = BlobHelper.deserializeBlob(bytes);
+
+    assertThat(object).isNotNull();
+    assertThat(object.getClass().getName()).isEqualTo(CLASS_NAME_SERIALIZABLE_IMPL);
+    assertThat(Serializable.class.isInstance(object));
+
+    Class deserializedClass = object.getClass();
+    assertThat(deserializedClass.getInterfaces()).contains(Serializable.class);
+  }
+
+  /**
+   * Tests that the deserialized object has the correct state
+   */
+  @Test
+  public void handlesObjectWithStateFromOtherClassLoader() throws Exception {
+    Class loadedClass = Class.forName(CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE, true, Thread.currentThread().getContextClassLoader());
+
+    Constructor ctor = loadedClass.getConstructor(new Class[] {Object.class});
+    Valuable instance = (Valuable) ctor.newInstance(new Object[] {123});
+    assertThat(instance.getValue()).isEqualTo(123);
+
+    byte[] bytes = BlobHelper.serializeToBlob(instance);
+
+    Valuable object = (Valuable) BlobHelper.deserializeBlob(bytes);
+    assertThat(object.getValue()).isEqualTo(instance.getValue());
+  }
+
+  /**
+   * Custom class loader which uses BCEL to dynamically generate SerializableImpl or SerializableImplWithValue.
+   */
+  private static class GeneratingClassLoader extends ClassLoader {
+
+    private static final String GENERATED = "<generated>";
+
+    private final Map<String, Class<?>> classDefinitions;
+
+    public GeneratingClassLoader(ClassLoader parent) {
+      super(parent);
+      this.classDefinitions = new HashMap<>();
+    }
+
+    public GeneratingClassLoader() {
+      this(null); // no parent
+    }
+
+    @Override
+    protected Class<?> findClass(String name) throws ClassNotFoundException {
+      Class<?> definedClass = null;
+      synchronized (this.classDefinitions) {
+        definedClass = getClass(name);
+        if (definedClass == null) {
+          definedClass = generate(name);
+          this.classDefinitions.put(name, definedClass);
+        }
+      }
+      return definedClass;
+    }
+
+    @Override
+    protected URL findResource(String name) {
+      return null;
+    }
+
+    @Override
+    protected Enumeration<URL> findResources(String name) throws IOException {
+      return null;
+    }
+
+    private Class<?> generate(String name) throws ClassNotFoundException {
+      if (CLASS_NAME_SERIALIZABLE_IMPL.equals(name)) {
+        return generateSerializableImpl();
+      } else if (CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE.equals(name)) {
+        return generateSerializableImplWithValue();
+      } else {
+        return null;
+        //throw new Error("Unable to generate " + name);
+      }
+    }
+
+    /**
+     * <pre>
+     * public class SerializableImpl implements Serializable {
+     *
+     *   public SerializableImpl() {
+     *   }
+     *
+     * }
+     * </pre>
+     */
+    private Class<?> generateSerializableImpl() throws ClassNotFoundException {
+      ClassGen cg = new ClassGen(CLASS_NAME_SERIALIZABLE_IMPL, Object.class.getName(), GENERATED, Constants.ACC_PUBLIC | Constants.ACC_SUPER, new String[] {Serializable.class.getName()});
+      cg.addEmptyConstructor(Constants.ACC_PUBLIC);
+      JavaClass jClazz = cg.getJavaClass();
+      byte[] bytes = jClazz.getBytes();
+      return defineClass(jClazz.getClassName(), bytes, 0, bytes.length);
+    }
+
+    /**
+     * <pre>
+     * public class SerializableImplWithValue extends SerializableImpl implements Valuable {
+     *
+     *   private Object value;
+     *
+     *   public SerializableImplWithValue() {
+     *   }
+     *
+     *   public SerializableImplWithValue(Object value) {
+     *     this.value = value;
+     *   }
+     *
+     *   public Object getValue() {
+     *     return this.value;
+     *   }
+     *
+     *   public void setValue(Object value) {
+     *     this.value = value;
+     *   }
+     * }
+     * </pre>
+     *
+     * @see Valuable
+     */
+    private Class<?> generateSerializableImplWithValue() throws ClassNotFoundException {
+      ClassGen cg = new ClassGen(CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE, CLASS_NAME_SERIALIZABLE_IMPL, GENERATED, Constants.ACC_PUBLIC | Constants.ACC_SUPER, new String[] {Valuable.class.getName()});
+      ConstantPoolGen cp = cg.getConstantPool();
+      InstructionFactory fac = new InstructionFactory(cg, cp);
+
+      // field
+      FieldGen fg = new FieldGen(Constants.ACC_PRIVATE, Type.OBJECT, VALUE, cp);
+      Field field = fg.getField();
+      cg.addField(field);
+
+      // empty constructor
+      cg.addEmptyConstructor(Constants.ACC_PUBLIC);
+
+      // constructor with arg
+      InstructionList ctor = new InstructionList();
+      MethodGen ctorMethod = new MethodGen(Constants.ACC_PUBLIC, Type.VOID, new Type[] { Type.OBJECT }, new String[] { "arg0" }, "<init>", "com.gemstone.gemfire.internal.util.bcel.SerializableImplWithValue", ctor, cp);
+      ctorMethod.setMaxStack(2);
+
+      InstructionHandle ctor_ih_0 = ctor.append(fac.createLoad(Type.OBJECT, 0));
+      ctor.append(fac.createInvoke(CLASS_NAME_SERIALIZABLE_IMPL, "<init>", Type.VOID, Type.NO_ARGS, Constants.INVOKESPECIAL));
+      InstructionHandle ctor_ih_4 = ctor.append(fac.createLoad(Type.OBJECT, 0));
+      ctor.append(fac.createLoad(Type.OBJECT, 1));
+      ctor.append(fac.createFieldAccess(CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE, "value", Type.OBJECT, Constants.PUTFIELD));
+      InstructionHandle ctor_ih_9 = ctor.append(fac.createReturn(Type.VOID));
+
+      cg.addMethod(ctorMethod.getMethod());
+      ctor.dispose();
+
+      // getter
+      InstructionList getter = new InstructionList();
+      MethodGen getterMethod = new MethodGen(Constants.ACC_PUBLIC, Type.OBJECT, null, null, GET_VALUE, CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE, getter, cp);
+      getterMethod.setMaxStack(1);
+
+      InstructionHandle getter_ih_0 = getter.append(fac.createLoad(Type.OBJECT, 0));
+      InstructionHandle getter_ih_1 = getter.append(fac.createGetField(cg.getClassName(), field.getName(), Type.getType(field.getSignature())));
+      InstructionHandle getter_ih_4 = getter.append(fac.createReturn(Type.OBJECT));
+
+      cg.addMethod(getterMethod.getMethod());
+      getter.dispose();
+
+      // setter
+      InstructionList setter = new InstructionList();
+      MethodGen setterMethod = new MethodGen(Constants.ACC_PUBLIC, Type.VOID, new Type[] {Type.OBJECT}, new String[] {field.getName()}, SET_VALUE, CLASS_NAME_SERIALIZABLE_IMPL_WITH_VALUE, setter, cp);
+      setterMethod.setMaxStack(2);
+
+      InstructionHandle setter_ih_0 = setter.append(fac.createLoad(Type.OBJECT, 0));
+      InstructionHandle setter_ih_1 = setter.append(fac.createLoad(Type.OBJECT, 1));
+      InstructionHandle setter_ih_2 = setter.append(fac.createPutField(cg.getClassName(), field.getName(), Type.getType(field.getSignature())));
+      InstructionHandle setter_ih_0_ih_5 = setter.append(fac.createReturn(Type.VOID));
+
+      cg.addMethod(setterMethod.getMethod());
+      setter.dispose();
+
+      JavaClass jClazz = cg.getJavaClass();
+      byte[] bytes = jClazz.getBytes();
+      return defineClass(jClazz.getClassName(), bytes, 0, bytes.length);
+    }
+
+    private Class<?> getClass(String name) {
+      synchronized (this.classDefinitions) {
+        return this.classDefinitions.get(name);
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec1415f4/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImpl.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImpl.java
deleted file mode 100644
index 44a8363..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImpl.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.util;
-
-import java.io.*;
-
-/**
- * A {@link Serializable} class that is loaded by a class loader other
- * than the one that is used to load test classes.
- *
- *
- * @since 2.0.1
- */
-public class SerializableImpl implements Serializable {
-
-  /**
-   * Creates a new <code>SerializableImpl</code>
-   */
-  public SerializableImpl() {
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec1415f4/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImplWithValue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImplWithValue.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImplWithValue.java
deleted file mode 100644
index 5727ff8..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/SerializableImplWithValue.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.util;
-
-//import java.io.*;
-
-/**
- * A {@link SerializableImpl} that implements an interface and has
- * some object state.
- *
- *
- * @since 2.0.3
- */
-public class SerializableImplWithValue extends SerializableImpl 
-  implements Valuable {
-
-  /** This object's state */
-  private Object value;
-
-  /**
-   * Zero-argument constructor
-   */
-  public SerializableImplWithValue() {
-
-  }
-
-  /**
-   * Creates a new <code>SerializableImplWithValue</code> with a given
-   * value
-   */
-  public SerializableImplWithValue(Object value) {
-    this.value = value;
-  }
-
-  public Object getValue() {
-    return this.value;
-  }
-
-  public void setValue(Object value) {
-    this.value = value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec1415f4/geode-core/src/test/java/com/gemstone/gemfire/internal/util/Valuable.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/Valuable.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/Valuable.java
index f58deb9..885b665 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/Valuable.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/Valuable.java
@@ -21,7 +21,6 @@ package com.gemstone.gemfire.internal.util;
  * value associated with them.  This interface is not considered to be
  * a "user class".
  *
- *
  * @since 2.0.3
  */
 public interface Valuable {


[48/50] [abbrv] incubator-geode git commit: GEODE-1243: Missed fixing one test for parameterization

Posted by kl...@apache.org.
GEODE-1243: Missed fixing one test for parameterization


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

Branch: refs/heads/feature/GEODE-835
Commit: f05e87d2710cf26bb546109962a99c849cef973e
Parents: 2dd3d90
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri May 20 08:11:09 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri May 20 08:11:09 2016 -0700

----------------------------------------------------------------------
 .../distributed/LocatorLauncherAssemblyIntegrationTest.java     | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f05e87d2/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
index 332cbf4..84d4aba 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyIntegrationTest.java
@@ -27,10 +27,13 @@ import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.management.ManagementService;
 import com.gemstone.gemfire.management.ManagerMXBean;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.io.File;
 
@@ -40,6 +43,8 @@ import static org.junit.Assert.*;
  * These tests are part of assembly as they require the REST war file to be present.
  */
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherAssemblyIntegrationTest extends AbstractLocatorLauncherIntegrationTestCase {
 
   @Before


[42/50] [abbrv] incubator-geode git commit: GEODE-11: Fixing some races in Lucene Query DUnits

Posted by kl...@apache.org.
GEODE-11: Fixing some races in Lucene Query DUnits

The tests in LuceneQueryBase had a couple of race conditions. The first
test didn't actually wait for entries to be flushed, so it could run the
query before the flush happens.

The wait for flush test had a test hook with a 1 second pause, but that
may not be long enough depending on what happens on the system. I
changed the test to pause the sender instead, for a deterministic test.


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

Branch: refs/heads/feature/GEODE-835
Commit: 4a97daf63627f3e1d464996167f3dbbdffe29ac2
Parents: 8d1fa0c
Author: Dan Smith <up...@apache.org>
Authored: Wed May 18 16:40:28 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu May 19 17:02:05 2016 -0700

----------------------------------------------------------------------
 .../lucene/internal/LuceneEventListener.java    |  9 ----
 ...IndexCreationPersistenceIntegrationTest.java |  7 +--
 .../gemfire/cache/lucene/LuceneQueriesBase.java | 57 +++++---------------
 .../cache/lucene/LuceneQueriesPRBase.java       |  2 +
 .../lucene/LuceneQueriesPeerPRDUnitTest.java    |  1 +
 .../LuceneQueriesPeerPROverflowDUnitTest.java   |  2 +
 .../cache/lucene/test/LuceneTestUtilities.java  | 11 ++++
 7 files changed, 31 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
index 2dae4ee..ca8077d 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
@@ -72,10 +72,6 @@ public class LuceneEventListener implements AsyncEventListener {
         IndexRepository repository = repositoryManager.getRepository(region, key, callbackArgument);
 
         Operation op = event.getOperation();
-        
-        if (testHook != null) {
-          testHook.doTestHook("FOUND_AND_BEFORE_PROCESSING_A_EVENT");
-        }
 
         if (op.isCreate()) {
           repository.create(key, event.getDeserializedValue());
@@ -102,9 +98,4 @@ public class LuceneEventListener implements AsyncEventListener {
       DefaultQuery.setPdxReadSerialized(false);
     }
   }
-  
-  public interface TestHook {
-    public void doTestHook(String spot);
-  }
-  public static TestHook testHook;
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
index 23983cb..d6bf116 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -22,23 +22,19 @@ import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
 import java.io.File;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
 import com.gemstone.gemfire.cache.lucene.test.TestObject;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
-import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.queryparser.classic.ParseException;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -101,8 +97,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
     //Pause the sender so that the entry stays in the queue
-    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue(cache);
-    queue.getSender().pause();
+    pauseSender(cache);
 
     dataRegion.put("A", new TestObject());
     cache.close();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index c7567f3..821be17 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -18,6 +18,7 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
 import java.io.Serializable;
@@ -48,8 +49,6 @@ import org.junit.Test;
   */
 public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
 
-  protected static final String INDEX_NAME = "index";
-  protected static final String REGION_NAME = "index";
   private static final long serialVersionUID = 1L;
   protected VM dataStore1;
   protected VM dataStore2;
@@ -78,6 +77,7 @@ public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
     accessor.invoke(() -> initAccessor(createIndex));
 
     putDataInRegion(accessor);
+    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
     executeTextSearch(accessor);
   }
 
@@ -91,57 +91,28 @@ public abstract class LuceneQueriesBase extends JUnit4CacheTestCase {
     dataStore2.invoke(() -> initDataStore(createIndex));
     accessor.invoke(() -> initAccessor(createIndex));
 
-    try {
-      dataStore1.invoke(() -> setTestHook());
-      putDataInRegion(accessor);
-      waitForFlushBeforeExecuteTextSearch(accessor, 10);
-      executeTextSearch(accessor);
-    } finally {
-      dataStore1.invoke(() -> checkResultAndresetTestHook());
-    }
+    //Pause the sender to make sure some entries are queued up
+    dataStore1.invoke(() -> pauseSender(getCache()));
+    dataStore2.invoke(() -> pauseSender(getCache()));
+    putDataInRegion(accessor);
+    assertFalse(waitForFlushBeforeExecuteTextSearch(accessor, 500));
+    dataStore1.invoke(() -> resumeSender(getCache()));
+    dataStore2.invoke(() -> resumeSender(getCache()));
+    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
+    executeTextSearch(accessor);
   }
 
-  protected void waitForFlushBeforeExecuteTextSearch(VM vm, final int expectKeyNum) {
-    vm.invoke(() -> {
+  protected boolean waitForFlushBeforeExecuteTextSearch(VM vm, int ms) {
+    return vm.invoke(() -> {
       Cache cache = getCache();
-      Region<Object, Object> region = cache.getRegion(REGION_NAME);
 
       LuceneService service = LuceneServiceProvider.get(cache);
       LuceneIndexImpl index = (LuceneIndexImpl)service.getIndex(INDEX_NAME, REGION_NAME);
-      assertNotNull(index);
-      LuceneQuery<Integer, TestObject> query;
 
-      String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX_NAME, REGION_NAME);
-      AsyncEventQueue queue = cache.getAsyncEventQueue(aeqId);
-      assertNotNull(queue);
-      assertTrue(queue.size()>0);
-      index.waitUntilFlushed(30000);
-      return null;
+      return index.waitUntilFlushed(ms);
     });
   }
 
-  public static void setTestHook() {
-    LuceneEventListener.testHook = new LuceneEventListener.TestHook() {
-
-      @Override
-      public void doTestHook(String spot) {
-        if (spot.equals("FOUND_AND_BEFORE_PROCESSING_A_EVENT")) {
-          try {
-            Thread.sleep(1000);
-            LogService.getLogger().debug("Waited in test hook");
-          }
-          catch (InterruptedException e) {
-          }
-        }
-      }
-    };
-  }
-  
-  public static void checkResultAndresetTestHook()
-  {
-    LuceneEventListener.testHook = null;
-  }
-
   protected void executeTextSearch(VM vm) {
     vm.invoke(() -> {
       Cache cache = getCache();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
index fbd101e..4d5a0b7 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPRBase.java
@@ -19,6 +19,7 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
 import static org.junit.Assert.*;
 
 import java.io.Serializable;
@@ -60,6 +61,7 @@ public abstract class LuceneQueriesPRBase extends LuceneQueriesBase {
     dataStore2.invoke(() -> initDataStore(createIndex));
 
     rebalanceRegion(dataStore1);
+    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
     executeTextSearch(accessor);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
index 51d0a33..830ca26 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPRDUnitTest.java
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
index cf2bac7..8fd0a08 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPeerPROverflowDUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+
 import com.gemstone.gemfire.cache.EvictionAction;
 import com.gemstone.gemfire.cache.EvictionAttributes;
 import com.gemstone.gemfire.cache.RegionShortcut;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4a97daf6/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 7a3ef04..571049c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -28,6 +28,7 @@ import java.util.stream.Collectors;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
@@ -82,4 +83,14 @@ public class LuceneTestUtilities {
     }
     assertEquals(expectedKeySet, actualKeySet);
   }
+
+  public static void pauseSender(final Cache cache) {
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue(cache);
+    queue.getSender().pause();
+  }
+
+  public static void resumeSender(final Cache cache) {
+    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue(cache);
+    queue.getSender().resume();
+  }
 }


[11/50] [abbrv] incubator-geode git commit: GEODE-493 CI failure: PartitionedRegionSingleHopDUnitTest.test_MetadataContents GEODE-699 PartitionedRegionSingleHopDUnitTest.test_MetadataServiceCallAccuracy

Posted by kl...@apache.org.
GEODE-493 CI failure:
PartitionedRegionSingleHopDUnitTest.test_MetadataContents
GEODE-699
PartitionedRegionSingleHopDUnitTest.test_MetadataServiceCallAccuracy

Add a counter fetchTaskCount in ClientMetadataService to count the
number fetch task threads. Make sure the counter is increased before a
fetch task thread is scheduled. And decrease the counter when a thread
is done.

In the test, use Awaitility to make all fetch tasks threads are
completed, i.e. fetchTaskCount == 0, before the next step.


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

Branch: refs/heads/feature/GEODE-835
Commit: 92805bbb97fb1bc62db906f61f89ba71ec66f2da
Parents: 08ef0c1
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Sat May 14 15:59:32 2016 -0700
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Sat May 14 15:59:32 2016 -0700

----------------------------------------------------------------------
 .../client/internal/ClientMetadataService.java  | 31 ++++++++++++++++++--
 .../PartitionedRegionSingleHopDUnitTest.java    |  8 +++--
 2 files changed, 34 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/92805bbb/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
index ed26708..60318b0 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ClientMetadataService.java
@@ -72,7 +72,13 @@ public final class ClientMetadataService {
   /** random number generator used in pruning */
   private final Random rand = new Random();
   
-  private volatile boolean isMetadataStable = true; 
+  private volatile boolean isMetadataStable = true;
+
+  private boolean isMetadataRefreshed_TEST_ONLY = false;
+  
+  private int fetchTaskCount = 0;
+  
+  private final Object fetchTaskCountLock = new Object();
   
   public ClientMetadataService(Cache cache) {
     this.cache = cache;
@@ -528,6 +534,9 @@ public final class ClientMetadataService {
       }
     }
     else {
+      synchronized (fetchTaskCountLock){
+        fetchTaskCount++;
+      }
       Runnable fetchTask = new Runnable() {
         @SuppressWarnings("synthetic-access")
         public void run() {
@@ -544,6 +553,11 @@ public final class ClientMetadataService {
               logger.debug("An exception occurred while fetching metadata", e);
             }
           }
+          finally {
+            synchronized (fetchTaskCountLock){
+              fetchTaskCount--;
+            }
+          }
         }
       };
       SingleHopClientExecutor.submitTask(fetchTask);
@@ -630,6 +644,9 @@ public final class ClientMetadataService {
         }
       }
     } else {
+      synchronized (fetchTaskCountLock){
+        fetchTaskCount++;
+      }
       Runnable fetchTask = new Runnable() {
         @SuppressWarnings("synthetic-access")
         public void run() {
@@ -644,6 +661,11 @@ public final class ClientMetadataService {
               logger.debug("An exception occurred while fetching metadata", e);
             }
           }
+          finally {
+            synchronized (fetchTaskCountLock){
+              fetchTaskCount--;
+            }
+          }
         }
       };
       SingleHopClientExecutor.submitTask(fetchTask);
@@ -841,6 +863,9 @@ public final class ClientMetadataService {
     this.isMetadataStable = isMetadataStable;
   }
 
-  private boolean isMetadataRefreshed_TEST_ONLY = false;
-
+  public int getFetchTaskCount() {
+    synchronized(fetchTaskCountLock) {
+      return fetchTaskCount;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/92805bbb/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
index f79d6c6..65cc7eb 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
@@ -342,7 +342,6 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
   // Put data, get data and make the metadata stable.
   // Now verify that metadata has all 8 buckets info.
   // Now update and ensure the fetch service is never called.
-  @Category(FlakyTest.class) // GEODE-493: random ports, waitForCriterions
   public void test_MetadataContents() {
     Integer port0 = (Integer)member0.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
     Integer port1 = (Integer)member1.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
@@ -368,7 +367,6 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
   // once,
   // fetchservice has to be triggered.
   // Now put again from c2.There should be no hop at all.
-  @Category(FlakyTest.class) // GEODE-699: random ports, async actions, time sensitive
   public void test_MetadataServiceCallAccuracy() {
     Integer port0 = (Integer)member0.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
     Integer port1 = (Integer)member1.invoke(() -> PartitionedRegionSingleHopDUnitTest.createServer( 1, 4 ));
@@ -391,6 +389,9 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
 
     Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> cms.isRefreshMetadataTestOnly() == true);
 
+    //make sure all fetch tasks are completed
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> cms.getFetchTaskCount() == 0);
+
     cms.satisfyRefreshMetadata_TEST_ONLY(false);
     region.put(new Integer(0), "create0");
     region.put(new Integer(1), "create1");
@@ -1973,6 +1974,9 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
 
   private void verifyMetadata() {
     ClientMetadataService cms = ((GemFireCacheImpl)cache).getClientMetadataService();
+    //make sure all fetch tasks are completed
+    Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> cms.getFetchTaskCount() == 0);
+
     final Map<String, ClientPartitionAdvisor> regionMetaData = cms
         .getClientPRMetadata_TEST_ONLY();
     Awaitility.waitAtMost(60, TimeUnit.SECONDS).until(() -> (regionMetaData.size() == 4));


[30/50] [abbrv] incubator-geode git commit: GEODE-1391 Update javadocs for JSONFormatter

Posted by kl...@apache.org.
GEODE-1391 Update javadocs for JSONFormatter


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

Branch: refs/heads/feature/GEODE-835
Commit: b3d9061eaf0b95f6c548c4201c403811a0a2290d
Parents: b1ff33d
Author: Hitesh Khamesra <hk...@pivotal.io>
Authored: Mon May 16 09:33:36 2016 -0700
Committer: Hitesh Khamesra <hk...@pivotal.io>
Committed: Wed May 18 09:36:14 2016 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/pdx/JSONFormatter.java | 45 +++++++++++---------
 1 file changed, 26 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b3d9061e/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java b/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
index d929cf6..b9b6329 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
@@ -17,8 +17,8 @@
 package com.gemstone.gemfire.pdx;
 
 import java.io.IOException;
+import java.math.BigDecimal;
 import java.math.BigInteger;
-
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.JsonParser;
@@ -32,20 +32,27 @@ import com.gemstone.gemfire.pdx.internal.json.PdxToJSON;
 
 
 /**
- * PdxFormatter class has static methods to convert JSON document into {@link PdxInstance}
- * and methods to convert back {@link PdxInstance} into JSON Document.
+ * PdxFormatter class has static method {@link JSONFormatter#fromJSON(String)} to convert JSON 
+ * document into a {@link PdxInstance} and static method {@link JSONFormatter#toJSON(PdxInstance)} 
+ * to convert back {@link PdxInstance} into a JSON Document.
  * 
- * Using this, application can put json document in gemfire cache. Application can define indexes 
- * on PdxInsatnce and then query those using OQL. Query will return the PdxInstances as results,
- * that needs to convert back into JSON document. 
+ * Using this, application can put PdxInstance(a converted JSON document) in a geode cache. 
+ * Application can define indexes on PdxInstance and then query those using OQL. Query will 
+ * return the PdxInstances as a result, that needs to convert back into JSON document. 
  * 
  * This uses Jackson parser to parse the json document. Parser treats values in json document as 
  * number(byte, short, int, long..), string, array, object, 'true', 'false' or 'null'. Which
- * further treated as corresponding java types in PdxInstance
+ * further treated as following java types in PdxInstance
  * 
- * JSON objects are converted into PdxInstance
- * JSON arrays are converted into List.
- *  
+ * JSON object is converted into {@link PdxInstance}
+ * JSON arrays is converted into {@link java.util.LinkedList}
+ * JSON BigDecimal is converted into {@link BigDecimal}
+ * JSON BigInterger is converted into {@link BigInteger}
+ * JSON Double is converted into java primitive double
+ * JSON float is converted into java primitive float 
+ * JSON boolean is converted into java primitive boolean
+ * JSON Integer is converted based on its range to java byte, short or int.
+ * JSON null is converted java null object.
  */
 
 public class JSONFormatter {
@@ -220,7 +227,7 @@ public class JSONFormatter {
         case VALUE_NULL :
         {
           //write null
-          nullfound(currentState);
+          nullFound(currentState);
           currentState = states.SCALER_FOUND;
           currentPdxInstance.addNullField(currentFieldName);
           currentFieldName = null;
@@ -229,7 +236,7 @@ public class JSONFormatter {
         case VALUE_NUMBER_FLOAT:
         {
           //write double/float
-          doublefound(currentState);
+          doubleFound(currentState);
           currentState = states.SCALER_FOUND;
           //currentPdxInstance.addDoubleField(currentFieldName, jp.getDoubleValue());
           setNumberField(jp, currentPdxInstance, currentFieldName);
@@ -239,7 +246,7 @@ public class JSONFormatter {
         case VALUE_NUMBER_INT:
         {
          //write int
-          doublefound(currentState);
+          intFound(currentState);
           currentState = states.SCALER_FOUND;
           //currentPdxInstance.addIntField(currentFieldName, jp.getIntValue());
           setNumberField(jp, currentPdxInstance, currentFieldName);
@@ -249,7 +256,7 @@ public class JSONFormatter {
         case VALUE_STRING:
         {
           //write string
-          doublefound(currentState);
+          stringFound(currentState);
           currentState = states.SCALER_FOUND;
           currentPdxInstance.addStringField(currentFieldName, new String(jp.getText()));
           currentFieldName = null;
@@ -431,7 +438,7 @@ public class JSONFormatter {
         case VALUE_NULL :
         {
           //write null
-          nullfound(currentState);
+          nullFound(currentState);
           currentState = states.SCALER_FOUND;
           currentPdxList.addNullField(null);          
           break;
@@ -439,7 +446,7 @@ public class JSONFormatter {
         case VALUE_NUMBER_FLOAT:
         {
           //write double/float
-          doublefound(currentState);
+          doubleFound(currentState);
           currentState = states.SCALER_FOUND;
           //currentPdxList.addDoubleField(jp.getDoubleValue());
           setNumberField(jp,currentPdxList);
@@ -448,7 +455,7 @@ public class JSONFormatter {
         case VALUE_NUMBER_INT:
         {
          //write int
-          doublefound(currentState);
+          intFound(currentState);
           currentState = states.SCALER_FOUND;
          // currentPdxList.addIntField(jp.getIntValue());
           setNumberField(jp,currentPdxList);
@@ -590,7 +597,7 @@ public class JSONFormatter {
     }
   }
   
-  private boolean doublefound(states currentState)
+  private boolean doubleFound(states currentState)
   {
     switch(currentState)
     {
@@ -622,7 +629,7 @@ public class JSONFormatter {
     }
   }
   
-  private boolean nullfound(states currentState)
+  private boolean nullFound(states currentState)
   {
     switch(currentState)
     {


[24/50] [abbrv] incubator-geode git commit: GEODE-1344: fix the testcase to remove the durable client part

Posted by kl...@apache.org.
GEODE-1344: fix the testcase to remove the durable client part

The test case for durable client with 2 cache servers on the same JVM is not
mature. Since we did not change any logic in durable client, remove the test
code for now. This senario will be followed up in GEODE-1102.


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

Branch: refs/heads/feature/GEODE-835
Commit: 8266f6b9b944adddeea449743f8da611f76d9852
Parents: 8024f33
Author: zhouxh <gz...@pivotal.io>
Authored: Tue May 17 14:13:55 2016 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Tue May 17 14:16:04 2016 -0700

----------------------------------------------------------------------
 .../cache/wan/Simple2CacheServerDUnitTest.java  | 47 ++++++++++++++++----
 1 file changed, 39 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8266f6b9/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/Simple2CacheServerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/Simple2CacheServerDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/Simple2CacheServerDUnitTest.java
index 4fdd88d..2b7617d 100755
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/Simple2CacheServerDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/Simple2CacheServerDUnitTest.java
@@ -17,22 +17,31 @@
 package com.gemstone.gemfire.internal.cache.wan;
 
 import java.util.Iterator;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
 import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.ha.HAContainerRegion;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.jayway.awaitility.Awaitility;
 
@@ -45,11 +54,6 @@ public class Simple2CacheServerDUnitTest extends WANTestBase {
     super(name);
   }
   
-  // GEODE-1183: random ports, failure to start threads, eats exceptions, time sensitive
-  public void testDurableClient2MultipleCacheServer() throws Exception {
-    doMultipleCacheServer(true);
-  }
-
   public void testNormalClient2MultipleCacheServer() throws Exception {
     doMultipleCacheServer(false);
   }
@@ -80,11 +84,38 @@ public class Simple2CacheServerDUnitTest extends WANTestBase {
     Awaitility.waitAtMost(20, TimeUnit.SECONDS).until(() -> { return checkProxyIsPrimary(vm0) || checkProxyIsPrimary(vm1); });
     
     // close the current primary cache server, then re-test
-    vm1.invoke(()-> CacheClientNotifierDUnitTest.closeACacheServer(serverPort2));
-    Awaitility.waitAtMost(20, TimeUnit.SECONDS).until(() -> { return checkProxyIsPrimary(vm0) || checkProxyIsPrimary(vm1); });
+    int serverPortAtVM1 = vm1.invoke(()-> findCacheServerForPrimaryProxy());
+    if (serverPortAtVM1 != 0) {
+      vm1.invoke(()-> CacheClientNotifierDUnitTest.closeACacheServer(serverPortAtVM1));
+      LogService.getLogger().info("Closed cache server on vm1:"+serverPortAtVM1);
+      Awaitility.waitAtMost(20, TimeUnit.SECONDS).until(() -> { return checkProxyIsPrimary(vm0) || checkProxyIsPrimary(vm1); });
+    } else {
+      vm0.invoke(()-> CacheClientNotifierDUnitTest.closeACacheServer(serverPort3));
+      LogService.getLogger().info("Closed cache server on vm0:"+serverPort3);
+      assertTrue(checkProxyIsPrimary(vm1));
+    }
     disconnectAllFromDS();
   }
-
+  
+  private static int findCacheServerForPrimaryProxy() {
+    List<CacheServer> cacheServers = ((GemFireCacheImpl)cache).getCacheServers();
+    CacheServerImpl server = null;
+    for (CacheServer cs:cacheServers) {
+      server = (CacheServerImpl)cs;
+      long acceptorId = server.getAcceptor().getAcceptorId();
+      for (CacheClientProxy proxy:CacheClientNotifier.getInstance().getClientProxies()) {
+        if (proxy.isPrimary() == false) {
+          continue;
+        }
+        if (proxy.getAcceptorId() == acceptorId) {
+          LogService.getLogger().info("Found cache server "+server+" for the primary proxy "+proxy);
+          return server.getPort();
+        }
+      }
+    }
+    return 0;
+  }
+  
   public static void setClientServerObserver()
   {
     PoolImpl.AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = true;


[33/50] [abbrv] incubator-geode git commit: added my gnupg key

Posted by kl...@apache.org.
added my gnupg key


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

Branch: refs/heads/feature/GEODE-835
Commit: 3a37159d93f586bb351526b2ef464d5814ca685b
Parents: ef8b8b6
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Wed May 18 13:47:14 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Wed May 18 13:48:10 2016 -0700

----------------------------------------------------------------------
 KEYS | 37 +++++++++++++++++++++++++++++++++++++
 1 file changed, 37 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3a37159d/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
index 150a54a..a72e8e6 100644
--- a/KEYS
+++ b/KEYS
@@ -541,3 +541,40 @@ fzLIZEDxJMC091/3hdJjqgDVMLsOtK23z8hzpfXs/IDB20c7RLVS4OXOP8KGo/9D
 B2iHcBxKUnI5dZQQVx6WMbWZDyRakvc+tw1JXvY=
 =gNaz
 -----END PGP PUBLIC KEY BLOCK-----
+pub   2048R/C7B8A8AD 2016-05-17 [expires: 2022-05-16]
+uid                  Bruce J Schuchardt (Apache signing key) <bs...@gmail.com>
+sig 3        C7B8A8AD 2016-05-17  Bruce J Schuchardt (Apache signing key) <bs...@gmail.com>
+sub   2048R/CF3D4FD6 2016-05-17 [expires: 2022-05-16]
+sig          C7B8A8AD 2016-05-17  Bruce J Schuchardt (Apache signing key) <bs...@gmail.com>
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+Version: GnuPG v1
+
+mQENBFc7WDsBCACtmj2wpwU+tuEPEa9xh67MlRH07GgjEmG4lNcvAjRGGnekqW6n
+nfl4Geqz9jqNUbCoYOdTQ7VFS3PZSuG3rLV00Ejb1fTNESgMip+6h5U8gVnwtFjf
+WZbWGNJi1qcyDAkNpSWAJiez/NiAim5wRZYPsX89NTHKN3QR8r3/Ks4O8wihuc9Q
+/SOH2CzrVdZNEUm3dvtlNiqdm29mzFzqWyIerOKzt8JdifDXfV3M6BJlJYnR1tuH
+yDQvZRe61YOddht5LLG0GW8SPIJqNCBpGfFw++dQZlKqaNoOLRt0gsMDCGxy+mG5
+dlnDdI03voCKjeKAA4FZhKAVceYa6AQE0NMPABEBAAG0P0JydWNlIEogU2NodWNo
+YXJkdCAoQXBhY2hlIHNpZ25pbmcga2V5KSA8YnNjaHVjaGFyZHRAZ21haWwuY29t
+PokBPgQTAQIAKAUCVztYOwIbAwUJC0c1AAYLCQgHAwIGFQgCCQoLBBYCAwECHgEC
+F4AACgkQv0BlPce4qK384gf/Sm4AuKcZk5+jBJo+K8UNmK1tayzXtYS/+XTIW0X9
+qWlu8HrUj99OsI5vAM509nDvgGdH2VSp5zXRufhieicrA/j2NSYqHU+ogvs53iSY
+Gyr03YJlVgr1padGgmDpGtQyEtBFRRvkVLTTVoWRWQoe1PAFCPT7XgjQTLIjFEBc
+684O5O98bgss8NWkPJrrxTs/8CUPtzmiUj31jeXMEqnQF/9jjOFDU1RXPSODU064
+IXgIFX8yNk7whrYU6/DaLVzkKGsdTgOkEj3vDqyyL7J4MvLwioG5b0EUdNNr+JJ6
+qklrgN6q3d0lDfVrEZdyofLuI9+jP7P6SwqNrpM36nwENLkBDQRXO1g7AQgAz7K3
+X88CehFUh9hyKlcvF7tAFbS6SIk0bFpejlJz+j7depthHTZYo4rqXYYkVqcvCeT1
+yjpUURRtkBjiaQFKucKs8gIp251EhtjHr3nWRYCD6zNqzySDJ6qtXbpfYUDH+w2b
+urtXt2glBVHiSvt9rUsHSm6bX3FOU9vqF0yaTl0jFBjkbypZvV9HQE/CYKDggb9Q
+WtEDJlCEISiwZJGwpRERMnIO0Qgf6/tMQ8aM6EOAzBk5Nxwkyzvpl+hIFwTOYn6o
+Cw8tsRj6/Lca2iaAwXbS21Aq55ccJT/gV+GEguZXcZC/lpZ27BBGNViFUBXjYAOL
+kIG6CR9qXtZzLy9L7wARAQABiQElBBgBAgAPBQJXO1g7AhsMBQkLRzUAAAoJEL9A
+ZT3HuKitZaYH/j9omUPI96UqjVrcN0pFI38QMi2ViAHClV53kvsUfrfhH3ug6fdM
+XZvv2j3Y1T1A+FW9GY+SuPYwgpYOYFZcUxGGZC+ROJCG0akxyxigzIhnqU5ed4jC
+mOzaqYvx2d24cNFp3Y1bxmImrrjIFRiQaY4RAJ0UWwSBhsQKfkW5iWpmcvdeXAYI
+Gar+SkbQCj9AI/AR9uSr4N9qaQGGZ08fhUbtqqaRsv59Hq2RIYQl0ZzhOrvRQv4+
+2rw5i6WX6JRRv+aH+Iloiz457Ev1i7j0xwtc20s0BqR1JxejJYPA8lPGmpF9y1Vj
+zTSobHHbqgy42ehDcPEEb8HpLTAx16RLvjQ=
+=o8Ov
+-----END PGP PUBLIC KEY BLOCK-----


[31/50] [abbrv] incubator-geode git commit: GEODE-1378: Don't launch dunit VMs in checkMissedTests

Posted by kl...@apache.org.
GEODE-1378: Don't launch dunit VMs in checkMissedTests

Setting a system property to disable launching dunit VMs during
checkMissedTests.

DistributedLockServiceDUnitTest failed with this property set because it
was doing work in the constructor. I cleaned up the test to create the
blackboard in the set up method instead.


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

Branch: refs/heads/feature/GEODE-835
Commit: 18761c729396936e1f05dcfd69f344f5762dedb7
Parents: b3d9061
Author: Dan Smith <up...@apache.org>
Authored: Tue May 17 15:00:53 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed May 18 11:20:28 2016 -0700

----------------------------------------------------------------------
 .../DistributedLockServiceDUnitTest.java        | 46 +++++++++-----------
 gradle/test.gradle                              |  4 ++
 2 files changed, 25 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/18761c72/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedLockServiceDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedLockServiceDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedLockServiceDUnitTest.java
index 42c3f01..5729bf6 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedLockServiceDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedLockServiceDUnitTest.java
@@ -54,10 +54,14 @@ import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.experimental.categories.Category;
 
 /**
  * This class tests distributed ownership via the DistributedLockService api.
  */
+@Category(DistributedTest.class)
 public class DistributedLockServiceDUnitTest extends DistributedTestCase {
   
 	protected static DistributedSystem dlstSystem;
@@ -72,25 +76,10 @@ public class DistributedLockServiceDUnitTest extends DistributedTestCase {
 
   public DistributedLockServiceDUnitTest(String name) {
     super(name);
-    if (blackboard == null) {
-      try {
-        blackboard = DistributedLockBlackboardImpl.getInstance();
-      } catch (Exception e) {
-        throw new RuntimeException("initialization error", e);
-      }
-    }
   }
   
   /////////// Test lifecycle //////////
- 
-  public static void caseSetUp() throws Exception {
-    disconnectAllFromDS();
-  }
-  
-  public static void caseTearDown() throws Exception {
-    disconnectAllFromDS();
-  }
-  
+
   /**
    * Returns a previously created (or new, if this is the first
    * time this method is called in this VM) distributed system
@@ -98,23 +87,25 @@ public class DistributedLockServiceDUnitTest extends DistributedTestCase {
    */
   @Override
   public final void postSetUp() throws Exception {
+
+    createBlackboard();
+    Invoke.invokeInEveryVM(() -> createBlackboard());
+
     // Create a DistributedSystem in every VM
     connectDistributedSystem();
 
-    for (int h = 0; h < Host.getHostCount(); h++) {
-      Host host = Host.getHost(h);
+    Invoke.invokeInEveryVM(() -> connectDistributedSystem());
+  }
 
-      for (int v = 0; v < host.getVMCount(); v++) {
-        host.getVM(v).invoke(
-          DistributedLockServiceDUnitTest.class, "connectDistributedSystem", null);
-      }
+  private void createBlackboard() throws Exception {
+    if (blackboard == null) {
+      blackboard = DistributedLockBlackboardImpl.getInstance();
     }
   }
 
   @Override
   public final void preTearDown() throws Exception {
-    Invoke.invokeInEveryVM(DistributedLockServiceDUnitTest.class,
-                    "destroyAllDLockServices"); 
+    Invoke.invokeInEveryVM(() -> destroyAllDLockServices());
 //    invokeInEveryVM(DistributedLockServiceDUnitTest.class,
 //                    "remoteDumpAllDLockServices"); 
                     
@@ -126,7 +117,12 @@ public class DistributedLockServiceDUnitTest extends DistributedTestCase {
     
     this.lockGrantor = null;
   }
-  
+
+  @Override
+  public void postTearDown() throws Exception {
+    disconnectAllFromDS();
+  }
+
   public static void destroyAllDLockServices() {
     DLockService.destroyAll();
     dlstSystem = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/18761c72/gradle/test.gradle
----------------------------------------------------------------------
diff --git a/gradle/test.gradle b/gradle/test.gradle
index 96fbfc5..1d3701f 100644
--- a/gradle/test.gradle
+++ b/gradle/test.gradle
@@ -80,6 +80,10 @@ subprojects {
       excludeCategories 'com.gemstone.gemfire.test.junit.categories.UITest'
     }
 
+    //Skip launching any DUnit VMs during this run. This will prevent
+    //junit from launching VMs while parsing categories
+    systemProperty 'gemfire.DUnitLauncher.LAUNCHED', 'true'
+
     beforeTest { descriptor ->
       throw new GradleException("The test " + descriptor.getClassName() + "." + descriptor.getName() + " does not include a junit category.");
     }


[13/50] [abbrv] incubator-geode git commit: GEODE-1252: modify bits field atomically

Posted by kl...@apache.org.
GEODE-1252: modify bits field atomically

- AtomicIntegerFieldUpdater now used to modify bits field
- added unit test for bit methods
- removed unused constructors


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

Branch: refs/heads/feature/GEODE-835
Commit: 442895462472c31fb3b9f37ae61f7fe510423bf5
Parents: 6523c97
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed May 11 14:38:09 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon May 16 11:24:13 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/versions/VersionTag.java     | 83 +++++++++++-------
 .../versions/AbstractVersionTagTestBase.java    | 92 ++++++++++++++++++++
 .../cache/versions/VMVersionTagTest.java        | 32 +++++++
 .../sanctionedDataSerializables.txt             |  4 +-
 4 files changed, 179 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44289546/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
index 60e4299..7910996 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/versions/VersionTag.java
@@ -19,6 +19,7 @@ package com.gemstone.gemfire.internal.cache.versions;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
 import org.apache.logging.log4j.Logger;
 
@@ -74,6 +75,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
   private static final int BITS_TIMESTAMP_APPLIED = 0x20;
 
   private static final int BITS_ALLOWED_BY_RESOLVER = 0x40;
+  // Note: the only valid BITS_* are 0xFFFF.
   
   /**
    * the per-entry version number for the operation
@@ -100,10 +102,19 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    */
   private byte distributedSystemId;
 
+  // In GEODE-1252 we found that the bits field
+  // was concurrently modified by calls to
+  // setPreviousMemberID and setRecorded.
+  // So bits has been changed to volatile and
+  // all modification to it happens using AtomicIntegerFieldUpdater.
+  private static final AtomicIntegerFieldUpdater<VersionTag> bitsUpdater =
+      AtomicIntegerFieldUpdater.newUpdater(VersionTag.class, "bits");
   /**
    * boolean bits
+   * Note: this is an int field so it has 32 bits BUT only the lower 16 bits are serialized.
+   * So all our code should treat this an an unsigned short field.
    */
-  private int bits;
+  private volatile int bits;
 
   /**
    * the initiator of the operation.  If null, the initiator was the sender
@@ -128,7 +139,11 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
 
   /** record that the timestamp from this tag was applied to the cache */
   public void setTimeStampApplied(boolean isTimeStampUpdated) {
-    this.bits |= BITS_TIMESTAMP_APPLIED;
+    if (isTimeStampUpdated) {
+      setBits(BITS_TIMESTAMP_APPLIED);
+    } else {
+      clearBits(~BITS_TIMESTAMP_APPLIED);
+    }
   }
 
   /**
@@ -152,9 +167,9 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
 
   public void setIsGatewayTag(boolean isGateway) {
     if (isGateway) {
-      this.bits = this.bits | BITS_GATEWAY_TAG;
+      setBits(BITS_GATEWAY_TAG);
     } else {
-      this.bits = this.bits & ~BITS_GATEWAY_TAG;
+      clearBits(~BITS_GATEWAY_TAG);
     }
   }
 
@@ -193,7 +208,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    * set that this tag has been recorded in a receiver's RVV
    */
   public void setRecorded() {
-    this.bits |= BITS_RECORDED;
+    setBits(BITS_RECORDED);
   }
 
   /**
@@ -236,7 +251,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    * @param previousMemberID the previousMemberID to set
    */
   public void setPreviousMemberID(T previousMemberID) {
-    this.bits |= BITS_HAS_PREVIOUS_ID;
+    setBits(BITS_HAS_PREVIOUS_ID);
     this.previousMemberID = previousMemberID;
   }
 
@@ -249,9 +264,9 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    */
   public VersionTag setPosDup(boolean flag) {
     if (flag) {
-      this.bits |= BITS_POSDUP;
+      setBits(BITS_POSDUP);
     } else {
-      this.bits &= ~BITS_POSDUP;
+      clearBits(~BITS_POSDUP);
     }
     return this;
   }
@@ -268,9 +283,9 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
    */
   public VersionTag setAllowedByResolver(boolean flag) {
     if (flag) {
-      this.bits |= BITS_ALLOWED_BY_RESOLVER;
+      setBits(BITS_ALLOWED_BY_RESOLVER);
     } else {
-      this.bits &= ~BITS_ALLOWED_BY_RESOLVER;
+      clearBits(~BITS_ALLOWED_BY_RESOLVER);
     }
     return this;
   }
@@ -319,21 +334,6 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
     return !(this.entryVersion == 0 && this.regionVersionHighBytes == 0 && this.regionVersionLowBytes == 0);
   }
 
-  public VersionTag() {
-  }
-
-  /**
-   * creates a version tag for a WAN gateway event
-   *
-   * @param timestamp
-   * @param dsid
-   */
-  public VersionTag(long timestamp, int dsid) {
-    this.timeStamp = timestamp;
-    this.distributedSystemId = (byte) (dsid & 0xFF);
-    this.bits = BITS_GATEWAY_TAG + BITS_IS_REMOTE_TAG;
-  }
-
   public void toData(DataOutput out) throws IOException {
     toData(out, true);
   }
@@ -386,7 +386,7 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
     if (logger.isTraceEnabled(LogMarker.VERSION_TAG)) {
       logger.info(LogMarker.VERSION_TAG, "deserializing {} with flags 0x{}", this.getClass(), Integer.toHexString(flags));
     }
-    this.bits = in.readUnsignedShort();
+    bitsUpdater.set(this, in.readUnsignedShort());
     this.distributedSystemId = in.readByte();
     if ((flags & VERSION_TWO_BYTES) != 0) {
       this.entryVersion = in.readShort() & 0xffff;
@@ -408,11 +408,11 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
         this.previousMemberID = readMember(in);
       }
     }
-    this.bits |= BITS_IS_REMOTE_TAG;
+    setIsRemoteForTesting();
   }
   
   public void setIsRemoteForTesting() {
-    this.bits |= BITS_IS_REMOTE_TAG;
+    setBits(BITS_IS_REMOTE_TAG);
   }
 
   public abstract T readMember(DataInput in) throws IOException, ClassNotFoundException;
@@ -440,14 +440,14 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
       if (this.memberID != null) {
         s.append("; mbr=").append(this.memberID);
       }
-      if ((this.bits & BITS_HAS_PREVIOUS_ID) != 0) {
+      if (hasPreviousMemberID()) {
         s.append("; prev=").append(this.previousMemberID);
       }
       if (this.distributedSystemId >= 0) {
         s.append("; ds=").append(this.distributedSystemId);
       }
       s.append("; time=").append(getVersionTimeStamp());
-      if ((this.bits & BITS_IS_REMOTE_TAG) != 0) {
+      if (isFromOtherMember()) {
         s.append("; remote");
       }
       if (this.isAllowedByResolver()) {
@@ -544,4 +544,27 @@ public abstract class VersionTag<T extends VersionSource> implements DataSeriali
     }
     return true;
   }
+  
+  /**
+   * Set any bits in the given bitMask on the bits field
+   */
+  private void setBits(int bitMask) {
+    int oldBits;
+    int newBits;
+    do {
+      oldBits = this.bits;
+      newBits = oldBits | bitMask;
+    } while (!bitsUpdater.compareAndSet(this, oldBits, newBits));
+  }
+  /**
+   * Clear any bits not in the given bitMask from the bits field
+   */
+  private void clearBits(int bitMask) {
+    int oldBits;
+    int newBits;
+    do {
+      oldBits = this.bits;
+      newBits = oldBits & bitMask;
+    } while (!bitsUpdater.compareAndSet(this, oldBits, newBits));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44289546/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java
new file mode 100644
index 0000000..bf0ce43
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/AbstractVersionTagTestBase.java
@@ -0,0 +1,92 @@
+/*
+ * 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.versions;
+
+import static org.junit.Assert.*;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public abstract class AbstractVersionTagTestBase {
+  @SuppressWarnings("rawtypes")
+  protected abstract VersionTag createVersionTag();
+  
+  @SuppressWarnings("rawtypes")
+  private VersionTag vt;
+  
+  @Before
+  public void setup() {
+    this.vt = createVersionTag();
+  }
+  @Test
+  public void testFromOtherMemberBit() {
+    assertEquals(false, vt.isFromOtherMember());
+    vt.setIsRemoteForTesting();
+    assertEquals(true, vt.isFromOtherMember());
+  }
+  
+  @Test
+  public void testTimeStampUpdatedBit() {
+    assertEquals(false, vt.isTimeStampUpdated());
+    vt.setTimeStampApplied(true);
+    assertEquals(true, vt.isTimeStampUpdated());
+    vt.setTimeStampApplied(false);
+    assertEquals(false, vt.isTimeStampUpdated());
+  }
+  
+  @Test
+  public void testGatewayTagBit() {
+    assertEquals(false, vt.isGatewayTag());
+    vt.setIsGatewayTag(true);
+    assertEquals(true, vt.isGatewayTag());
+    vt.setIsGatewayTag(false);
+    assertEquals(false, vt.isGatewayTag());
+  }
+  
+  @Test
+  public void testRecordedBit() {
+    assertEquals(false, vt.isRecorded());
+    vt.setRecorded();
+    assertEquals(true, vt.isRecorded());
+  }
+  
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testPreviousMemberIDBit() {
+    assertEquals(false, vt.hasPreviousMemberID());
+    vt.setPreviousMemberID(null);
+    assertEquals(true, vt.hasPreviousMemberID());
+  }
+  
+  @Test
+  public void testPosDupBit() {
+    assertEquals(false, vt.isPosDup());
+    vt.setPosDup(true);
+    assertEquals(true, vt.isPosDup());
+    vt.setPosDup(false);
+    assertEquals(false, vt.isPosDup());
+  }
+  
+  @Test
+  public void testAllowedByResolverBit() {
+    assertEquals(false, vt.isAllowedByResolver());
+    vt.setAllowedByResolver(true);
+    assertEquals(true, vt.isAllowedByResolver());
+    vt.setAllowedByResolver(false);
+    assertEquals(false, vt.isAllowedByResolver());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44289546/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java
new file mode 100644
index 0000000..4e39f3d
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/versions/VMVersionTagTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.versions;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class VMVersionTagTest extends AbstractVersionTagTestBase {
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  protected VersionTag createVersionTag() {
+    return new VMVersionTag();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/44289546/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index fe21fbf..d2204a0 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -1975,8 +1975,8 @@ fromData,212,2a2a2bb600bdb500132bb900be01003d2a1c047e04a0000704a7000403b500972ab
 toData,239,2ab4001499000dbb00b05912b8b700b2bf2a2ab400132bb600b9033d2ab400979900071c04803d2b1cb900ba02002b2ab4000bb60035b900bb03002b2ab4000cb60035b900bb03002b2ab40025b60033b900ba02002ab40025b60099b9002b01004e2db9002c010099002b2db9002d0100c0002e3a042a1904b9002f0100c0006e2bb600b91904b9003001002bb800bca7ffd22b2ab40026b60033b900ba02002ab40026b60099b9002b01004e2db9002c01009900332db9002d0100c0002e3a042a1904b9002f0100c0006e2bb600b92b1904b900300100c0003eb6003fb900bb0300a7ffca2ab4001f2bb800bcb1
 
 com/gemstone/gemfire/internal/cache/versions/VersionTag,2
-fromData,201,2bb9002001003db20012b20013b900140200990022b20012b20013122105bd001659032ab600175359041cb8001853b9001904002a2bb900200100b500012a2bb900220100b5000d1c077e9900132a2bb900230100121c7eb50003a7000f2a2bb900240100027eb500031c10107e99000d2a2bb900230100b500072a2bb900240100b5000a2a2bb80025b500041c047e99000c2a2a2bb60026b5000b1c057e99001e1c10087e99000e2a2ab4000bb5000ca7000c2a2a2bb60026b5000c2a59b40001101080b50001b1
-toData,269,033e0336042ab400031211a2000a0436041d07803e2ab400079900081d1010803e2ab4000bc6000b1c9900071d04803e2ab4000cc6001b1d05803e2ab4000c2ab4000ba6000c1c9900081d1008803eb20012b20013b900140200990022b20012b20013121505bd001659032ab600175359041db8001853b9001904002b1db9001a02002b2ab40001b9001a02002b2ab4000db9001b020015049900132b2ab40003121c7eb9001a0200a7000d2b2ab40003b9001d02002ab4000799000d2b2ab40007b9001a02002b2ab4000ab9001d02002ab400042bb8001e2ab4000bc600101c99000c2a2ab4000b2bb6001f2ab4000cc6001b2ab4000c2ab4000ba600071c9a000c2a2ab4000c2bb6001fb1
+fromData,197,2bb9002201003db20014b20015b900160200990022b20014b20015122305bd001859032ab600195359041cb8001a53b9001b0400b200242a2bb900220100b600252a2bb900260100b500101c077e9900132a2bb900270100121e7eb50006a7000f2a2bb900280100027eb500061c10107e99000d2a2bb900270100b5000a2a2bb900280100b5000d2a2bb80029b500071c047e99000c2a2a2bb6002ab5000e1c057e99001e1c10087e99000e2a2ab4000eb5000fa7000c2a2a2bb6002ab5000f2ab6002bb1
+toData,269,033e0336042ab400061213a2000a0436041d07803e2ab4000a9900081d1010803e2ab4000ec6000b1c9900071d04803e2ab4000fc6001b1d05803e2ab4000f2ab4000ea6000c1c9900081d1008803eb20014b20015b900160200990022b20014b20015121705bd001859032ab600195359041db8001a53b9001b04002b1db9001c02002b2ab40002b9001c02002b2ab40010b9001d020015049900132b2ab40006121e7eb9001c0200a7000d2b2ab40006b9001f02002ab4000a99000d2b2ab4000ab9001c02002b2ab4000db9001f02002ab400072bb800202ab4000ec600101c99000c2a2ab4000e2bb600212ab4000fc6001b2ab4000f2ab4000ea600071c9a000c2a2ab4000f2bb60021b1
 
 com/gemstone/gemfire/internal/cache/wan/GatewaySenderAdvisor$GatewaySenderProfile,4
 fromData,282,2a2bb700082a2bb80009b5000a2a2bb9000b0100b5000c2a2bb9000d0100b5000e2a2bb9000f0100b500102a2bb9000f0100b500112a2bb9000f0100b500122a2bb9000f0100b500132a2bb9000f0100b500142a2bb9000d0100b500152a2bb9000f0100b500162a2bb80017b500042a2bb80017b500052a2bb80017b500062a2bb9000f0100b500182a2bb9000d0100b500192bb8001ab2001bb6001c9c00552bb8001dc0001e4d2cc600412cb6001fb20020b60021b6002299000d2ab20020b50023a7002c2cb6001fb20024b60021b6002299000d2ab20024b50023a700122ab20025b50023a700082a01b50023a7000e2a2bb8001dc00026b500232bb800273d1c9900162abb002859b70029b5002a2ab4002a2bb8002bb1


[35/50] [abbrv] incubator-geode git commit: GEODE-1243: Locator can start on ephemeral port

Posted by kl...@apache.org.
GEODE-1243: Locator can start on ephemeral port


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

Branch: refs/heads/feature/GEODE-835
Commit: 6f4a879304b957696e220e7b806b65cfd09e4636
Parents: d14af16
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 4 13:06:52 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed May 18 14:39:17 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/distributed/Locator.java   |   9 +-
 .../gemfire/distributed/LocatorLauncher.java    |   4 +
 .../distributed/internal/InternalLocator.java   |  49 +++++----
 .../internal/tcpserver/TcpServer.java           |  41 ++++---
 .../cli/commands/LauncherLifecycleCommands.java |   2 +-
 ...tractLocatorLauncherIntegrationTestCase.java |  21 +++-
 .../distributed/HostedLocatorsDUnitTest.java    | 110 +++++++++++++++++++
 .../gemfire/distributed/LocatorJUnitTest.java   |  65 +++++++----
 ...LocatorLauncherLocalFileIntegrationTest.java |  10 +-
 .../LocatorLauncherLocalIntegrationTest.java    |  24 +++-
 ...ocatorLauncherRemoteFileIntegrationTest.java |   5 +-
 .../LocatorLauncherRemoteIntegrationTest.java   |  29 +++++
 .../InternalDistributedSystemJUnitTest.java     |  30 +----
 .../management/LocatorManagementDUnitTest.java  |  63 ++++++++++-
 14 files changed, 344 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/main/java/com/gemstone/gemfire/distributed/Locator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/Locator.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/Locator.java
index 80a0248..f089013 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/Locator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/Locator.java
@@ -73,9 +73,6 @@ public abstract class Locator {
   /** The file to which this locator logs */
   protected File logFile;
 
-  /** The port on which this locator listens */
-  protected int port;
-
   /** The bind address for this locator */
   protected InetAddress bindAddress;
 
@@ -423,9 +420,7 @@ public abstract class Locator {
   /**
    * Returns the port on which this locator runs
    */
-  public int getPort() {
-    return this.port;
-  }
+  public abstract Integer getPort();
 
   /**
    * Returns the distributed system started by this locator, if any
@@ -504,7 +499,7 @@ public abstract class Locator {
       }
     }
     StringBuilder locatorString = new StringBuilder(String.valueOf(ba));
-    locatorString.append('[').append(this.port).append(']');
+    locatorString.append('[').append(this.getPort()).append(']');
     return locatorString.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
index 24b859c..b9edb59 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/LocatorLauncher.java
@@ -408,6 +408,10 @@ public final class LocatorLauncher extends AbstractLauncher<String> {
    * @return an Integer value indicating the port number on which the Locator is listening for client requests.
    */
   public Integer getPort() {
+    if (locator != null) {
+      return locator.getPort();
+    }
+
     return this.port;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
index 7ad57ad..cf8d91e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
@@ -27,13 +27,11 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Properties;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import com.gemstone.gemfire.InternalGemFireException;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelException;
@@ -186,8 +184,7 @@ public class InternalLocator extends Locator implements ConnectListener {
   private volatile boolean isSharedConfigurationStarted = false; 
   
   private volatile Thread restartThread;
-  
-  
+
   public boolean isSharedConfigurationEnabled() {
     return this.config.getEnableClusterConfiguration();
   }
@@ -528,7 +525,6 @@ public class InternalLocator extends Locator implements ConnectListener {
     InetAddress bindAddress,
     String hostnameForClients,
     java.util.Properties distributedSystemProperties, DistributionConfigImpl cfg, boolean startDistributedSystem) {
-    this.port = port;
     this.logFile = logF;
     this.bindAddress = bindAddress;
     this.hostnameForClients = hostnameForClients;
@@ -551,8 +547,6 @@ public class InternalLocator extends Locator implements ConnectListener {
           bindAddress.getHostAddress());
     }
     
-    
-
     if (distributedSystemProperties != null) {
       env.putAll(distributedSystemProperties);
     }
@@ -607,14 +601,15 @@ public class InternalLocator extends Locator implements ConnectListener {
     
     this.locatorListener = WANServiceProvider.createLocatorMembershipListener();
     if(locatorListener != null) {
-      this.locatorListener.setPort(this.port);
+      // We defer setting the port until the handler is init'd - that way we'll have an actual port in the
+      // case where we're starting with port = 0.
       this.locatorListener.setConfig(this.getConfig());
     }
-    this.handler = new PrimaryHandler(this.port, this, locatorListener);
+    this.handler = new PrimaryHandler(this, locatorListener);
   
     ThreadGroup group = LoggingThreadGroup.createThreadGroup("Distribution locators", logger);
     stats = new LocatorStats();
-    server = new TcpServer(this.port, this.bindAddress, null, this.config,
+    server = new TcpServer(port, this.bindAddress, null, this.config,
         this.handler, new DelayedPoolStatHelper(), group, this.toString());
   }
 
@@ -735,7 +730,7 @@ public class InternalLocator extends Locator implements ConnectListener {
         else {
           sb.append(SocketCreator.getLocalHost().getHostAddress());
         }
-        sb.append('[').append(port).append(']');
+        sb.append('[').append(getPort()).append(']');
         thisLocator = sb.toString();
       }
       
@@ -844,7 +839,7 @@ public class InternalLocator extends Locator implements ConnectListener {
     
     this.locatorDiscoverer = WANServiceProvider.createLocatorDiscoverer();
     if(this.locatorDiscoverer != null) {
-      this.locatorDiscoverer.discover(this.port, config, locatorListener);
+      this.locatorDiscoverer.discover(getPort(), config, locatorListener);
     }
   }
   
@@ -875,7 +870,7 @@ public class InternalLocator extends Locator implements ConnectListener {
 
     this.productUseLog.monitorUse(distributedSystem);
     
-    ServerLocator sl = new ServerLocator(this.port, 
+    ServerLocator sl = new ServerLocator(getPort(),
                                          this.bindAddress,
                                          this.hostnameForClients,
                                          this.logFile,
@@ -957,7 +952,7 @@ public class InternalLocator extends Locator implements ConnectListener {
     if (this.server.isAlive()) {
       logger.info(LocalizedMessage.create(LocalizedStrings.InternalLocator_STOPPING__0, this));
       try {
-        stopLocator(this.port, this.bindAddress);
+        stopLocator(getPort(), this.bindAddress);
       } catch ( ConnectException ignore ) {
         // must not be running
       }
@@ -1234,7 +1229,19 @@ public class InternalLocator extends Locator implements ConnectListener {
   public ServerLocator getServerLocatorAdvisee() {
     return this.serverLocator;
   }
-  
+
+  /**
+   * Return the port on which the locator is actually listening. If called before the locator has actually
+   * started, this method will return null.
+   *
+   * @return the port the locator is listening on or null if it has not yet been started
+   */
+  public Integer getPort() {
+    if (server != null) {
+      return server.getPort();
+    }
+    return null;
+  }
   
   /******
    * 
@@ -1290,19 +1297,23 @@ public class InternalLocator extends Locator implements ConnectListener {
     private final LocatorMembershipListener locatorListener;
     //private final List<LocatorJoinMessage> locatorJoinMessages;
     private Object locatorJoinObject = new Object();
-    InternalLocator interalLocator;
+    private InternalLocator internalLocator;
     boolean willHaveServerLocator;  // flag to avoid warning about missing handlers during startup
     
-    public PrimaryHandler(int port, InternalLocator locator,
+    public PrimaryHandler(InternalLocator locator,
         LocatorMembershipListener listener) {
       this.locatorListener = listener;
-      interalLocator = locator;
+      internalLocator = locator;
       //this.locatorJoinMessages = new ArrayList<LocatorJoinMessage>();
     }
 
     // this method is synchronized to make sure that no new handlers are added while
     //initialization is taking place.
     public synchronized void init(TcpServer tcpServer) {
+      if (this.locatorListener != null) {
+        // This is deferred until now as the initial requested port could have been 0
+        this.locatorListener.setPort(internalLocator.getPort());
+      }
       this.tcpServer = tcpServer;
       for(Iterator itr = allHandlers.iterator(); itr.hasNext();) {
         TcpHandler handler = (TcpHandler) itr.next();
@@ -1364,7 +1375,7 @@ public class InternalLocator extends Locator implements ConnectListener {
         handler.shutDown();
       }
       } finally {
-        this.interalLocator.handleShutdown();
+        this.internalLocator.handleShutdown();
       }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
index c8c6f50..476636a 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/tcpserver/TcpServer.java
@@ -182,31 +182,22 @@ public class TcpServer {
   
   public void start() throws IOException {
     this.shuttingDown = false;
-    handler.init(this);
     startServerThread();
+    handler.init(this);
   }
   
   private void startServerThread() throws IOException {
     if (srv_sock == null || srv_sock.isClosed()) {
       if (bind_address == null) {
-        srv_sock = SocketCreator.getDefaultInstance().createServerSocket(port,
-            BACKLOG);
-        // srv_sock=new ServerSocket(port, 20); // backlog of 20 connections
+        srv_sock = SocketCreator.getDefaultInstance().createServerSocket(port, BACKLOG);
         bind_address = srv_sock.getInetAddress();
-      } else
-        srv_sock = SocketCreator.getDefaultInstance().createServerSocket(port,
-            BACKLOG, bind_address);
-
-        // TODO:GEODE-1243: srv_Sock now has a real port even though this.port is still zero -- update it or use a second var
-
-      // srv_sock=new ServerSocket(port, 20, bind_address); // backlog of 20
-      // connections
-      {
-        if (log.isInfoEnabled())
-          log.info("Locator was created at " + new Date());
-        if (log.isInfoEnabled())
-          log.info("Listening on port " + port + " bound on address "
-              + bind_address);
+      } else {
+        srv_sock = SocketCreator.getDefaultInstance().createServerSocket(port, BACKLOG, bind_address);
+      }
+
+      if (log.isInfoEnabled()) {
+        log.info("Locator was created at " + new Date());
+        log.info("Listening on port " + getPort() + " bound on address " + bind_address);
       }
       srv_sock.setReuseAddress(true); // GemStoneAddition
     }
@@ -247,6 +238,20 @@ public class TcpServer {
     return srv_sock.getLocalSocketAddress(); 
   }
 
+  /**
+   * Returns the value of the bound port. If the server was initialized with a port of 0 indicating that any
+   * ephemeral port should be used, this method will return the actual bound port.
+   *
+   * @return the port bound to this socket or 0 if the socket is closed or otherwise not connected
+   */
+  public int getPort() {
+    if (srv_sock != null && !srv_sock.isClosed()) {
+      return srv_sock.getLocalPort();
+    }
+
+    return 0;
+  }
+
   protected void run() {
     Socket sock = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
index 5b2c872..ceb6cb9 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
@@ -432,7 +432,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
         infoResultData.addLine(locatorState.toString());
 
         String locatorHostName = StringUtils.defaultIfBlank(locatorLauncher.getHostnameForClients(), getLocalHost());
-        int locatorPort = locatorLauncher.getPort();
+        int locatorPort = Integer.parseInt(locatorState.getPort());
 
         // AUTO-CONNECT
         // If the connect succeeds add the connected message to the result,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLocatorLauncherIntegrationTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLocatorLauncherIntegrationTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLocatorLauncherIntegrationTestCase.java
index 5434a01..d419b61 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLocatorLauncherIntegrationTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLocatorLauncherIntegrationTestCase.java
@@ -16,7 +16,10 @@
  */
 package com.gemstone.gemfire.distributed;
 
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.concurrent.Callable;
+import java.util.function.IntSupplier;
 
 import org.junit.After;
 import org.junit.Before;
@@ -30,13 +33,26 @@ import com.gemstone.gemfire.distributed.LocatorLauncher.LocatorState;
 import com.gemstone.gemfire.distributed.internal.SharedConfiguration;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.DistributionLocator;
+import org.junit.runners.Parameterized;
 
 /**
  * @since 8.0
  */
 public abstract class AbstractLocatorLauncherIntegrationTestCase extends AbstractLauncherIntegrationTestCase {
 
+  @Parameterized.Parameters
+  public static Collection<Object> data() {
+    return Arrays.asList(new Object[] {
+        (IntSupplier) () -> 0,
+        (IntSupplier) () -> AvailablePortHelper.getRandomAvailableTCPPort()
+    });
+  }
+
+  @Parameterized.Parameter
+  public IntSupplier portSupplier;
+
   protected volatile int locatorPort;
+
   protected volatile LocatorLauncher launcher;
   protected volatile String workingDirectory;
   protected volatile String clusterConfigDirectory;
@@ -49,9 +65,8 @@ public abstract class AbstractLocatorLauncherIntegrationTestCase extends Abstrac
 
   @Before
   public final void setUpAbstractLocatorLauncherIntegrationTestCase() throws Exception {
-    final int port = AvailablePortHelper.getRandomAvailableTCPPort();
-    System.setProperty(DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, String.valueOf(port));
-    this.locatorPort = port;
+    this.locatorPort = portSupplier.getAsInt();
+    System.setProperty(DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, String.valueOf(this.locatorPort));
     this.workingDirectory = this.temporaryFolder.getRoot().getCanonicalPath();
     this.clusterConfigDirectory = this.temporaryFolder.newFolder(SharedConfiguration.CLUSTER_CONFIG_DISK_DIR_PREFIX + getUniqueName()).getCanonicalPath();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
index 042e896..fe8c204 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/HostedLocatorsDUnitTest.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
+import com.gemstone.gemfire.test.dunit.Host;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -174,6 +175,115 @@ public class HostedLocatorsDUnitTest extends JUnit4DistributedTestCase {
     }
   }
 
+  @Test
+  public void testGetAllHostedLocatorsUsingPortZero() throws Exception {
+    final InternalDistributedSystem system = getSystem();
+    final String dunitLocator = system.getConfig().getLocators();
+    assertNotNull(dunitLocator);
+    assertFalse(dunitLocator.isEmpty());
+
+    // This will eventually contain the ports used by locators
+    final int[] ports = new int[] {0, 0, 0, 0};
+
+    final String uniqueName = getUniqueName();
+    for (int i = 0 ; i < 4; i++) {
+      final int whichvm = i;
+      Integer port = (Integer) Host.getHost(0).getVM(whichvm).invoke(new SerializableCallable() {
+        @Override
+        public Object call() throws Exception {
+          try {
+            System.setProperty("gemfire.locators", dunitLocator);
+            System.setProperty("gemfire.mcast-port", "0");
+
+            final String name = uniqueName + "-" + whichvm;
+            final File subdir = new File(name);
+            subdir.mkdir();
+            assertTrue(subdir.exists() && subdir.isDirectory());
+
+            final Builder builder = new Builder()
+                .setMemberName(name)
+                .setPort(ports[whichvm])
+                .setRedirectOutput(true)
+                .setWorkingDirectory(name);
+
+            launcher = builder.build();
+            assertEquals(Status.ONLINE, launcher.start().getStatus());
+            waitForLocatorToStart(launcher, TIMEOUT_MILLISECONDS, 10, true);
+            return launcher.getPort();
+          } finally {
+            System.clearProperty("gemfire.locators");
+            System.clearProperty("gemfire.mcast-port");
+          }
+        }
+      });
+      ports[i] = port;
+    }
+
+    final String host = SocketCreator.getLocalHost().getHostAddress();
+
+    final Set<String> locators = new HashSet<String>();
+    locators.add(host + "[" + dunitLocator.substring(dunitLocator.indexOf("[")+1, dunitLocator.indexOf("]")) + "]");
+    for (int port : ports) {
+      locators.add(host +"[" + port + "]");
+    }
+
+    // validation within non-locator
+    final DistributionManager dm = (DistributionManager)system.getDistributionManager();
+
+    final Set<InternalDistributedMember> locatorIds = dm.getLocatorDistributionManagerIds();
+    assertEquals(5, locatorIds.size());
+
+    final Map<InternalDistributedMember, Collection<String>> hostedLocators = dm.getAllHostedLocators();
+    assertTrue(!hostedLocators.isEmpty());
+    assertEquals(5, hostedLocators.size());
+
+    for (InternalDistributedMember member : hostedLocators.keySet()) {
+      assertEquals(1, hostedLocators.get(member).size());
+      final String hostedLocator = hostedLocators.get(member).iterator().next();
+      assertTrue(locators + " does not contain " + hostedLocator, locators.contains(hostedLocator));
+    }
+
+    // validate fix for #46324
+    for (int whichvm = 0 ; whichvm < 4; whichvm++) {
+      Host.getHost(0).getVM(whichvm).invoke(new SerializableRunnable() {
+        @Override
+        public void run() {
+          final DistributionManager dm = (DistributionManager)InternalDistributedSystem.getAnyInstance().getDistributionManager();
+          final InternalDistributedMember self = dm.getDistributionManagerId();
+
+          final Set<InternalDistributedMember> locatorIds = dm.getLocatorDistributionManagerIds();
+          assertTrue(locatorIds.contains(self));
+
+          final Map<InternalDistributedMember, Collection<String>> hostedLocators = dm.getAllHostedLocators();
+          assertTrue("hit bug #46324: " + hostedLocators + " is missing " + InternalLocator.getLocatorStrings() + " for " + self, hostedLocators.containsKey(self));
+        }
+      });
+    }
+
+    // validation with locators
+    for (int whichvm = 0 ; whichvm < 4; whichvm++) {
+      Host.getHost(0).getVM(whichvm).invoke(new SerializableRunnable() {
+        @Override
+        public void run() {
+          final DistributionManager dm = (DistributionManager)InternalDistributedSystem.getAnyInstance().getDistributionManager();
+
+          final Set<InternalDistributedMember> locatorIds = dm.getLocatorDistributionManagerIds();
+          assertEquals(5, locatorIds.size());
+
+          final Map<InternalDistributedMember, Collection<String>> hostedLocators = dm.getAllHostedLocators();
+          assertTrue(!hostedLocators.isEmpty());
+          assertEquals(5, hostedLocators.size());
+
+          for (InternalDistributedMember member : hostedLocators.keySet()) {
+            assertEquals(1, hostedLocators.get(member).size());
+            final String hostedLocator = hostedLocators.get(member).iterator().next();
+            assertTrue(locators + " does not contain " + hostedLocator, locators.contains(hostedLocator));
+          }
+        }
+      });
+    }
+  }
+
   protected void waitForLocatorToStart(final LocatorLauncher launcher, int timeout, int interval, boolean throwOnTimeout) throws Exception {
     assertEventuallyTrue("waiting for process to start: " + launcher.status(), new Callable<Boolean>() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
index 7ee4694..1d7afae 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
@@ -24,10 +24,15 @@ import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
+import java.util.function.IntSupplier;
 
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -50,15 +55,29 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.management.internal.JmxManagerAdvisor.JmxManagerProfile;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
 public class LocatorJUnitTest {
 
   private static final int REQUEST_TIMEOUT = 5 * 1000;
 
   private Locator locator;
-  private int port;
   private File tmpFile;
+  private int port;
+
+  @Parameterized.Parameters
+  public static Collection<Object> data() {
+    return Arrays.asList(new Object[] {
+        (IntSupplier) () -> 0,
+        (IntSupplier) () -> AvailablePortHelper.getRandomAvailableTCPPort()
+    });
+  }
+
+  @Parameterized.Parameter
+  public IntSupplier portSupplier;
 
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -66,8 +85,8 @@ public class LocatorJUnitTest {
   @Before
   public void setUp() throws IOException {
     tmpFile = File.createTempFile("locator", ".log");
-    port = getRandomAvailablePort(SOCKET);
-    File locatorFile = new File("locator"+port+".dat");
+    this.port = portSupplier.getAsInt();
+    File locatorFile = new File("locator" + this.port + ".dat");
     if (locatorFile.exists()) {
       locatorFile.delete();
     }
@@ -89,7 +108,6 @@ public class LocatorJUnitTest {
     Properties dsprops = new Properties();
     int jmxPort = getRandomAvailablePort(SOCKET);
     dsprops.setProperty("mcast-port", "0");
-    dsprops.setProperty("locators", "localhost[" + port + "]");
     dsprops.setProperty("jmx-manager-port", ""+jmxPort);
     dsprops.setProperty("jmx-manager-start", "true");
     dsprops.setProperty("jmx-manager-http-port", "0");
@@ -108,18 +126,18 @@ public class LocatorJUnitTest {
   @Test
   public void testBasicInfo() throws Exception {
     locator = Locator.startLocator(port, tmpFile);
-   assertTrue(locator.isPeerLocator());
-   assertFalse(locator.isServerLocator());
-    String[] info = InternalLocator.getLocatorInfo(InetAddress.getLocalHost(), port);
-   assertNotNull(info);
-   assertTrue(info.length > 1);
+    assertTrue(locator.isPeerLocator());
+    assertFalse(locator.isServerLocator());
+    int boundPort = (port == 0) ? locator.getPort() : port;
+    String[] info = InternalLocator.getLocatorInfo(InetAddress.getLocalHost(), boundPort);
+    assertNotNull(info);
+    assertTrue(info.length > 1);
   }
 
   @Test
   public void testNoThreadLeftBehind() throws Exception {
     Properties dsprops = new Properties();
     dsprops.setProperty("mcast-port", "0");
-    dsprops.setProperty("locators", "localhost[" + port + "]");
     dsprops.setProperty("jmx-manager-start", "false");
     dsprops.setProperty(ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
 
@@ -137,9 +155,9 @@ public class LocatorJUnitTest {
         }
       }
       if (threadCount < Thread.activeCount()) {
-          OSProcess.printStacks(0);
-          fail("expected " + threadCount + " threads or fewer but found " + Thread.activeCount()
-              +".  Check log file for a thread dump.");
+        OSProcess.printStacks(0);
+        fail("expected " + threadCount + " threads or fewer but found " + Thread.activeCount()
+            +".  Check log file for a thread dump.");
         }
     }
   }
@@ -153,7 +171,7 @@ public class LocatorJUnitTest {
    assertFalse(locator.isPeerLocator());
    assertTrue(locator.isServerLocator());
     Thread.sleep(1000);
-    doServerLocation();
+    doServerLocation(locator.getPort());
   }
 
   @Test
@@ -163,10 +181,10 @@ public class LocatorJUnitTest {
     props.setProperty(ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
 
     locator = Locator.startLocatorAndDS(port, tmpFile, null, props);
-   assertTrue(locator.isPeerLocator());
-   assertTrue(locator.isServerLocator());
+    assertTrue(locator.isPeerLocator());
+    assertTrue(locator.isServerLocator());
     Thread.sleep(1000);
-    doServerLocation();
+    doServerLocation(locator.getPort());
     locator.stop();
   }
 
@@ -185,19 +203,18 @@ public class LocatorJUnitTest {
     }
   }
 
-  private void doServerLocation() throws Exception {
+  private void doServerLocation(int realPort) throws Exception {
     {
       ClientConnectionRequest request = new ClientConnectionRequest(Collections.EMPTY_SET, "group1");
-      ClientConnectionResponse response = (ClientConnectionResponse) TcpClient.requestToServer(InetAddress.getLocalHost(), port, request, REQUEST_TIMEOUT);
-     assertEquals(null, response.getServer());
+      ClientConnectionResponse response = (ClientConnectionResponse) TcpClient.requestToServer(InetAddress.getLocalHost(), realPort, request, REQUEST_TIMEOUT);
+      assertEquals(null, response.getServer());
     }
 
     {
       QueueConnectionRequest request = new QueueConnectionRequest(ClientProxyMembershipID.getNewProxyMembership(InternalDistributedSystem.getAnyInstance()), 3, Collections.EMPTY_SET, "group1",true);
-      QueueConnectionResponse response = (QueueConnectionResponse) TcpClient.requestToServer(InetAddress.getLocalHost(), port, request, REQUEST_TIMEOUT);
-     assertEquals(new ArrayList(), response.getServers());
-     assertFalse(response.isDurableQueueFound());
+      QueueConnectionResponse response = (QueueConnectionResponse) TcpClient.requestToServer(InetAddress.getLocalHost(), realPort, request, REQUEST_TIMEOUT);
+      assertEquals(new ArrayList(), response.getServers());
+      assertFalse(response.isDurableQueueFound());
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
index c695b07..422fb7c 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
@@ -16,15 +16,16 @@
  */
 package com.gemstone.gemfire.distributed;
 
-import static org.junit.Assert.*;
-
+import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
-import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import static org.junit.Assert.assertFalse;
 
 /**
  * Subclass of LocatorLauncherLocalDUnitTest which forces the code to not find 
@@ -34,6 +35,7 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
  * @since 8.0
  */
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
 public class LocatorLauncherLocalFileIntegrationTest extends LocatorLauncherLocalIntegrationTest {
 
   @Before

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
index 897d0fa..9137207 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
@@ -43,6 +43,8 @@ import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
 import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 /**
  * Tests usage of LocatorLauncher as a local API in existing JVM.
@@ -50,8 +52,9 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
  * @since 8.0
  */
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
 public class LocatorLauncherLocalIntegrationTest extends AbstractLocatorLauncherIntegrationTestCase {
-  
+
   @Before
   public final void setUpLocatorLauncherLocalIntegrationTest() throws Exception {
     disconnectFromDS();
@@ -320,6 +323,11 @@ public class LocatorLauncherLocalIntegrationTest extends AbstractLocatorLauncher
   
   @Test
   public void testStartWithDefaultPortInUseFails() throws Throwable {
+    // Test makes no sense in this case
+    if (this.locatorPort == 0) {
+      return;
+    }
+
     this.socket = SocketCreator.getDefaultInstance().createServerSocket(this.locatorPort, 50, null, -1);
     assertTrue(this.socket.isBound());
     assertFalse(this.socket.isClosed());
@@ -480,13 +488,13 @@ public class LocatorLauncherLocalIntegrationTest extends AbstractLocatorLauncher
     }
   } // testStartWithExistingPidFileFails
   */
-  
+
   @Test
   public void testStartUsingPort() throws Throwable {
     // generate one free port and then use it instead of default
     final int freeTCPPort = AvailablePortHelper.getRandomAvailableTCPPort();
     assertTrue(AvailablePort.isPortAvailable(freeTCPPort, AvailablePort.SOCKET));
-    
+
     this.launcher = new Builder()
         .setMemberName(getUniqueName())
         .setPort(freeTCPPort)
@@ -531,13 +539,17 @@ public class LocatorLauncherLocalIntegrationTest extends AbstractLocatorLauncher
   
   @Test
   public void testStartUsingPortInUseFails() throws Throwable {
+    // Test makes no sense in this case
+    if (this.locatorPort == 0) {
+      return;
+    }
+
     // generate one free port and then use it instead of default
-    final int freeTCPPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    this.socket = SocketCreator.getDefaultInstance().createServerSocket(freeTCPPort, 50, null, -1);
+    this.socket = SocketCreator.getDefaultInstance().createServerSocket(this.locatorPort, 50, null, -1);
     
     this.launcher = new Builder()
         .setMemberName(getUniqueName())
-        .setPort(freeTCPPort)
+        .setPort(this.locatorPort)
         .setRedirectOutput(true)
         .setWorkingDirectory(this.workingDirectory)
         .set(DistributionConfig.CLUSTER_CONFIGURATION_DIR, this.clusterConfigDirectory)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
index 63f7312..13e936d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
@@ -35,6 +35,8 @@ import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.lang.AttachAPINotFoundException;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 /**
  * Subclass of LocatorLauncherRemoteDUnitTest which forces the code to not find 
@@ -44,8 +46,9 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
  * @since 8.0
  */
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
 public class LocatorLauncherRemoteFileIntegrationTest extends LocatorLauncherRemoteIntegrationTest {
-  
+
   @Before
   public final void setUpLocatorLauncherRemoteFileIntegrationTest() throws Exception {
     System.setProperty(ProcessControllerFactory.PROPERTY_DISABLE_ATTACH_API, "true");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
index a5b8831..508c13c 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
@@ -29,6 +29,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -48,6 +50,8 @@ import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 /**
  * Integration tests for launching a Locator in a forked process.
@@ -55,8 +59,31 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
  * @since 8.0
  */
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
 public class LocatorLauncherRemoteIntegrationTest extends AbstractLocatorLauncherRemoteIntegrationTestCase {
+
+  protected volatile Process process;
+  protected volatile ProcessStreamReader processOutReader;
+  protected volatile ProcessStreamReader processErrReader;
   
+  @Before
+  public final void setUpLocatorLauncherRemoteTest() throws Exception {
+  }
+
+  @After
+  public final void tearDownLocatorLauncherRemoteTest() throws Exception {
+    if (this.process != null) {
+      this.process.destroy();
+      this.process = null;
+    }
+    if (this.processOutReader != null && this.processOutReader.isRunning()) {
+      this.processOutReader.stop();
+    }
+    if (this.processErrReader != null && this.processErrReader.isRunning()) {
+      this.processErrReader.stop();
+    }
+  }
+
   @Test
   public void testIsAttachAPIFound() throws Exception {
     final ProcessControllerFactory factory = new ProcessControllerFactory();
@@ -368,6 +395,7 @@ public class LocatorLauncherRemoteIntegrationTest extends AbstractLocatorLaunche
   @Test
   public void testStartUsingPortInUseFails() throws Throwable {
     this.socket = SocketCreator.getDefaultInstance().createServerSocket(this.locatorPort, 50, null, -1);
+    this.locatorPort = this.socket.getLocalPort();
     
     final List<String> jvmArguments = getJvmArguments();
     
@@ -437,6 +465,7 @@ public class LocatorLauncherRemoteIntegrationTest extends AbstractLocatorLaunche
     AtomicBoolean outputContainedExpectedString = new AtomicBoolean();
 
     this.socket = SocketCreator.getDefaultInstance().createServerSocket(this.locatorPort, 50, null, -1);
+    this.locatorPort = this.socket.getLocalPort();
     
     assertFalse(AvailablePort.isPortAvailable(this.locatorPort, AvailablePort.SOCKET));
     assertTrue(this.socket.isBound());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
index 8832459..827091f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystemJUnitTest.java
@@ -108,8 +108,6 @@ public class InternalDistributedSystemJUnitTest
   @Test
   public void testDefaultProperties() {
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -165,8 +163,6 @@ public class InternalDistributedSystemJUnitTest
 
     Properties props = new Properties();
     props.put(DistributionConfig.NAME_NAME, name);
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -309,8 +305,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetLogLevel() {
     Level logLevel = Level.FINER;
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -336,8 +330,6 @@ public class InternalDistributedSystemJUnitTest
   @Test
   public void testGetStatisticSamplingEnabled() {
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -350,8 +342,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetStatisticSampleRate() {
     String rate = String.valueOf(DistributionConfig.MIN_STATISTIC_SAMPLE_RATE);
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -391,8 +381,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetStatisticArchiveFile() {
     String fileName = "testGetStatisticArchiveFile";
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -409,8 +397,6 @@ public class InternalDistributedSystemJUnitTest
   public void _testGetAckWaitThreshold() {
     String time = String.valueOf(DistributionConfig.MIN_ACK_WAIT_THRESHOLD);
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -440,8 +426,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetCacheXmlFile() {
     String fileName = "blah";
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -454,8 +438,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetArchiveDiskSpaceLimit() {
     String value = String.valueOf(DistributionConfig.MIN_ARCHIVE_DISK_SPACE_LIMIT);
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -481,8 +463,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetArchiveFileSizeLimit() {
     String value = String.valueOf(DistributionConfig.MIN_ARCHIVE_FILE_SIZE_LIMIT);
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -508,8 +488,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetLogDiskSpaceLimit() {
     String value = String.valueOf(DistributionConfig.MIN_LOG_DISK_SPACE_LIMIT);
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -535,8 +513,6 @@ public class InternalDistributedSystemJUnitTest
   public void testGetLogFileSizeLimit() {
     String value = String.valueOf(DistributionConfig.MIN_LOG_FILE_SIZE_LIMIT);
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -562,8 +538,6 @@ public class InternalDistributedSystemJUnitTest
   public void testAccessingClosedDistributedSystem() {
     Properties props = new Properties();
 
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -659,8 +633,6 @@ public class InternalDistributedSystemJUnitTest
     Level level = Level.FINE;
 
     Properties props = new Properties();
-//     int unusedPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-//     props.setProperty("mcast-port", String.valueOf(unusedPort));
     // a loner is all this test needs
     props.setProperty("mcast-port", "0");
     props.setProperty("locators", "");
@@ -684,7 +656,7 @@ public class InternalDistributedSystemJUnitTest
     Assert.assertTrue(locator.isPeerLocator());
 //    Assert.assertFalse(locator.isServerLocator()); server location is forced on while licensing is disabled in GemFire
 //    Assert.assertIndexDetailsEquals("127.0.0.1", locator.getBindAddress().getHostAddress());  removed this check for ipv6 testing
-    Assert.assertEquals(unusedPort, locator.getPort());
+    Assert.assertEquals(unusedPort, locator.getPort().intValue());
     deleteStateFile(unusedPort);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6f4a8793/geode-core/src/test/java/com/gemstone/gemfire/management/LocatorManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/LocatorManagementDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/LocatorManagementDUnitTest.java
index 7399ef8..fb8ae7d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/LocatorManagementDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/LocatorManagementDUnitTest.java
@@ -105,6 +105,28 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
 
   }
 
+  public void testPeerLocationWithPortZero() throws Exception {
+    // Start the locator with port=0
+    int locPort = startLocator(locator, true, 0);
+    locatorMBeanExist(locator, locPort, true);
+
+    Host host = Host.getHost(0);
+    String host0 = getServerHostName(host);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, host0 + "[" + locPort
+        + "]");
+    props.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
+    props.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "false");
+    props.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.JMX_MANAGER_HTTP_PORT_NAME, "0");
+    createCache(managingNode, props);
+    startManagingNode(managingNode);
+    DistributedMember locatorMember = getMember(locator);
+    remoteLocatorMBeanExist(managingNode,locatorMember);
+
+  }
+
   /**
    * Tests a locator which is co-located with already existing cache
    * 
@@ -118,6 +140,13 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
 
   }
 
+  public void testColocatedLocatorWithPortZero() throws Exception {
+    initManagement(false);
+    int locPort = startLocator(locator, false, 0);
+    locatorMBeanExist(locator, locPort, false);
+
+  }
+
   public void testListManagers() throws Exception {
     initManagement(false);
     int locPort = AvailablePortHelper.getRandomAvailableTCPPort();
@@ -125,6 +154,12 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
     listManagers(locator, locPort, false);
   }
 
+  public void testListManagersWithPortZero() throws Exception {
+    initManagement(false);
+    int locPort = startLocator(locator, false, 0);
+    listManagers(locator, locPort, false);
+  }
+
   public void testWillingManagers() throws Exception {
     int locPort = AvailablePortHelper.getRandomAvailableTCPPort();
     startLocator(locator, true, locPort);
@@ -144,6 +179,23 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
     listWillingManagers(locator, locPort, false);
   }
 
+  public void testWillingManagersWithPortZero() throws Exception {
+    int locPort = startLocator(locator, true, 0);
+
+    Host host = Host.getHost(0);
+    String host0 = getServerHostName(host);
+
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, host0 + "[" + locPort
+        + "]");
+    props.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
+
+    createCache(managedNode2, props);
+    createCache(managedNode3, props);
+
+    listWillingManagers(locator, locPort, false);
+  }
 
   /**
    * Starts a locator with given configuration.
@@ -152,10 +204,9 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
    * @param vm
    *          reference to VM
    */
-  protected String startLocator(final VM vm, final boolean isPeer,
-      final int port) {
+  protected Integer startLocator(final VM vm, final boolean isPeer, final int port) {
 
-    return (String) vm.invoke(new SerializableCallable("Start Locator In VM") {
+    return (Integer) vm.invoke(new SerializableCallable("Start Locator In VM") {
 
       public Object call() throws Exception {
 
@@ -174,16 +225,16 @@ public class LocatorManagementDUnitTest extends ManagementTestBase {
           Assert.fail("While resolving bind address ", uhe);
         }
 
+        Locator locator = null;
         try {
           File logFile = new File(getTestMethodName() + "-locator" + port + ".log");
-          Locator locator = Locator.startLocatorAndDS(port, logFile, bindAddr,
-              props, isPeer, true, null);
+          locator = Locator.startLocatorAndDS(port, logFile, bindAddr, props, isPeer, true, null);
         } catch (IOException ex) {
           Assert.fail("While starting locator on port " + port, ex);
         }
 
         assertTrue(InternalLocator.hasLocator());
-        return null;
+        return locator.getPort();
       }
     });
   }


[02/50] [abbrv] incubator-geode git commit: Revert "GEODE-1376: Cleaned up server port to be '0'."

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
index 692ff19..2e80ebf 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/rest/internal/web/controllers/RestAPIsWithSSLDUnitTest.java
@@ -19,6 +19,7 @@ package com.gemstone.gemfire.rest.internal.web.controllers;
 import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.cache.client.internal.LocatorTestBase;
 import com.gemstone.gemfire.cache.server.CacheServer;
@@ -30,10 +31,7 @@ import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.management.ManagementException;
 import com.gemstone.gemfire.management.ManagementTestBase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
-import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.util.test.TestUtil;
 import org.apache.http.HttpEntity;
 import org.apache.http.client.methods.CloseableHttpResponse;
@@ -49,12 +47,13 @@ import javax.net.ssl.SSLContext;
 import java.io.*;
 import java.net.BindException;
 import java.security.KeyStore;
-import java.util.Date;
+import java.util.Calendar;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
 /**
+ * 
  * @since 8.0
  */
 public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
@@ -64,13 +63,13 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
   private ManagementTestBase helper;
 
   private final String PEOPLE_REGION_NAME = "People";
-
+  
   private File jks;
 
   public RestAPIsWithSSLDUnitTest(String name) {
     super(name);
     this.helper = new ManagementTestBase(name);
-    this.jks = findTrustedJKS();
+    this.jks  = findTrustedJKS();
 
   }
 
@@ -86,22 +85,32 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
   }
 
   private File findTrustedJKS() {
-    if (jks == null) {
+    if(jks == null){
       jks = new File(TestUtil.getResourcePath(RestAPIsWithSSLDUnitTest.class, "/ssl/trusted.keystore"));
     }
     return jks;
   }
 
+  public String startBridgeServerWithRestServiceOnInVM(final VM vm, final String locators, final String[] regions,
+      final Properties sslProperties, final boolean clusterLevel) {
+
+    final String hostName = vm.getHost().getHostName();
+    final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    vm.invoke("startBridge", () -> startBridgeServer(hostName,serverPort,locators,regions,sslProperties,clusterLevel));
+    return "https://" + hostName + ":" + serverPort + "/gemfire-api/v1";
+
+  }
+
   @SuppressWarnings("deprecation")
-  protected int startBridgeServer(String hostName, int restServicePort, final String locators, final String[] regions,
+  protected int startBridgeServer(String hostName, int restServicerPort, final String locators, final String[] regions,
       final Properties sslProperties, boolean clusterLevel) {
 
     Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
     props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
     props.setProperty(DistributionConfig.START_DEV_REST_API_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_BIND_ADDRESS_NAME, hostName);
-    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(restServicePort));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_PORT_NAME, String.valueOf(restServicerPort));
 
     System.setProperty("javax.net.debug", "ssl,handshake");
     configureSSL(props, sslProperties, clusterLevel);
@@ -109,8 +118,8 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     DistributedSystem ds = getSystem(props);
     Cache cache = CacheFactory.create(ds);
     ((GemFireCacheImpl) cache).setReadSerialized(true);
-
     AttributesFactory factory = new AttributesFactory();
+
     factory.setEnableBridgeConflation(true);
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
@@ -119,45 +128,81 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     }
 
     CacheServer server = cache.addCacheServer();
-    server.setPort(0);
+    final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    server.setPort(serverPort);
     try {
       server.start();
     } catch (IOException e) {
       e.printStackTrace();
     }
     remoteObjects.put(CACHE_KEY, cache);
-    return new Integer(server.getPort());
+    return new Integer(serverPort);
   }
 
   public void doPutsInClientCache() {
-    ClientCache clientCache = ClientCacheFactory.getAnyInstance();
-    assertNotNull(clientCache);
-    Region<String, Object> region = clientCache.getRegion(PEOPLE_REGION_NAME);
+    ClientCache cache = GemFireCacheImpl.getInstance();
+    assertNotNull(cache);
+    Region<String, Object> region = cache.getRegion(PEOPLE_REGION_NAME);
 
     // put person object
-    region.put("1", new Person(101L, "Mithali", "Dorai", "Raj", new Date(), Gender.FEMALE));
-    region.put("2", new Person(102L, "Sachin", "Ramesh", "Tendulkar", new Date(), Gender.MALE));
-    region.put("3", new Person(103L, "Saurabh", "Baburav", "Ganguly", new Date(), Gender.MALE));
-    region.put("4", new Person(104L, "Rahul", "subrymanyam", "Dravid", new Date(), Gender.MALE));
-    region.put("5", new Person(105L, "Jhulan", "Chidambaram", "Goswami", new Date(), Gender.FEMALE));
+    final Person person1 = new Person(101L, "Mithali", "Dorai", "Raj", DateTimeUtils.createDate(1982,
+        Calendar.DECEMBER, 4), Gender.FEMALE);
+    final Person person2 = new Person(102L, "Sachin", "Ramesh", "Tendulkar", DateTimeUtils.createDate(1975,
+        Calendar.DECEMBER, 14), Gender.MALE);
+    final Person person3 = new Person(103L, "Saurabh", "Baburav", "Ganguly", DateTimeUtils.createDate(1972,
+        Calendar.AUGUST, 29), Gender.MALE);
+    final Person person4 = new Person(104L, "Rahul", "subrymanyam", "Dravid", DateTimeUtils.createDate(1979,
+        Calendar.MARCH, 17), Gender.MALE);
+    final Person person5 = new Person(105L, "Jhulan", "Chidambaram", "Goswami", DateTimeUtils.createDate(1983,
+        Calendar.NOVEMBER, 25), Gender.FEMALE);
+
+    region.put("1", person1);
+    region.put("2", person2);
+    region.put("3", person3);
+    region.put("4", person4);
+    region.put("5", person5);
+
+    final Person person6 = new Person(101L, "Rahul", "Rajiv", "Gndhi",
+        DateTimeUtils.createDate(1970, Calendar.MAY, 14), Gender.MALE);
+    final Person person7 = new Person(102L, "Narendra", "Damodar", "Modi", DateTimeUtils.createDate(1945,
+        Calendar.DECEMBER, 24), Gender.MALE);
+    final Person person8 = new Person(103L, "Atal", "Bihari", "Vajpayee", DateTimeUtils.createDate(1920,
+        Calendar.AUGUST, 9), Gender.MALE);
+    final Person person9 = new Person(104L, "Soniya", "Rajiv", "Gandhi", DateTimeUtils.createDate(1929, Calendar.MARCH,
+        27), Gender.FEMALE);
+    final Person person10 = new Person(104L, "Priyanka", "Robert", "Gandhi", DateTimeUtils.createDate(1973,
+        Calendar.APRIL, 15), Gender.FEMALE);
+
+    final Person person11 = new Person(104L, "Murali", "Manohar", "Joshi", DateTimeUtils.createDate(1923,
+        Calendar.APRIL, 25), Gender.MALE);
+    final Person person12 = new Person(104L, "Lalkrishna", "Parmhansh", "Advani", DateTimeUtils.createDate(1910,
+        Calendar.JANUARY, 01), Gender.MALE);
+    final Person person13 = new Person(104L, "Shushma", "kumari", "Swaraj", DateTimeUtils.createDate(1943,
+        Calendar.AUGUST, 10), Gender.FEMALE);
+    final Person person14 = new Person(104L, "Arun", "raman", "jetly", DateTimeUtils.createDate(1942, Calendar.OCTOBER,
+        27), Gender.MALE);
+    final Person person15 = new Person(104L, "Amit", "kumar", "shah", DateTimeUtils.createDate(1958, Calendar.DECEMBER,
+        21), Gender.MALE);
+    final Person person16 = new Person(104L, "Shila", "kumari", "Dixit", DateTimeUtils.createDate(1927,
+        Calendar.FEBRUARY, 15), Gender.FEMALE);
 
     Map<String, Object> userMap = new HashMap<String, Object>();
-    userMap.put("6", new Person(101L, "Rahul", "Rajiv", "Gndhi", new Date(), Gender.MALE));
-    userMap.put("7", new Person(102L, "Narendra", "Damodar", "Modi", new Date(), Gender.MALE));
-    userMap.put("8", new Person(103L, "Atal", "Bihari", "Vajpayee", new Date(), Gender.MALE));
-    userMap.put("9", new Person(104L, "Soniya", "Rajiv", "Gandhi", new Date(), Gender.FEMALE));
-    userMap.put("10", new Person(104L, "Priyanka", "Robert", "Gandhi", new Date(), Gender.FEMALE));
-    userMap.put("11", new Person(104L, "Murali", "Manohar", "Joshi", new Date(), Gender.MALE));
-    userMap.put("12", new Person(104L, "Lalkrishna", "Parmhansh", "Advani", new Date(), Gender.MALE));
-    userMap.put("13", new Person(104L, "Shushma", "kumari", "Swaraj", new Date(), Gender.FEMALE));
-    userMap.put("14", new Person(104L, "Arun", "raman", "jetly", new Date(), Gender.MALE));
-    userMap.put("15", new Person(104L, "Amit", "kumar", "shah", new Date(), Gender.MALE));
-    userMap.put("16", new Person(104L, "Shila", "kumari", "Dixit", new Date(), Gender.FEMALE));
+    userMap.put("6", person6);
+    userMap.put("7", person7);
+    userMap.put("8", person8);
+    userMap.put("9", person9);
+    userMap.put("10", person10);
+    userMap.put("11", person11);
+    userMap.put("12", person12);
+    userMap.put("13", person13);
+    userMap.put("14", person14);
+    userMap.put("15", person15);
+    userMap.put("16", person16);
 
     region.putAll(userMap);
 
-    if (clientCache != null)
-      clientCache.getLogger().info("Gemfire Cache Client: Puts successfully done");
+    if (cache != null)
+      cache.getLogger().info("Gemfire Cache Client: Puts successfully done");
 
   }
 
@@ -172,30 +217,24 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     // start locator
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
 
-    locator.invoke("Start Locator", () -> startLocator(locator.getHost(), locatorPort, ""));
+    startLocatorInVM(locator, locatorPort, "");
 
     // find locators
     String locators = NetworkUtils.getServerHostName(locator.getHost()) + "[" + locatorPort + "]";
 
     // start manager (peer cache)
-    manager.invoke("StartManager", () -> startManager(locators, new String[] { REGION_NAME }, sslProperties));
+    startManagerInVM(manager, locators, new String[] { REGION_NAME }, sslProperties);
 
     // start startBridgeServer With RestService enabled
-    String restEndpoint = server.invoke("startBridgeServerWithRestServiceOnInVM", () -> {
-      final String hostName = server.getHost().getHostName();
-      final int restServicePort = AvailablePortHelper.getRandomAvailableTCPPort();
-      startBridgeServer(hostName, restServicePort, locators, new String[] { REGION_NAME }, sslProperties, clusterLevel);
-      return "https://" + hostName + ":" + restServicePort + "/gemfire-api/v1";
-    });
+    String restEndpoint = startBridgeServerWithRestServiceOnInVM(server, locators, new String[] { REGION_NAME },
+        sslProperties, clusterLevel);
 
     // create a client cache
-    client.invoke("Create ClientCache", () -> new ClientCacheFactory()
-        .setPdxReadSerialized(true)
-        .addPoolLocator(NetworkUtils.getServerHostName(locator.getHost()), locatorPort).create());
+    createClientCacheInVM(client, NetworkUtils.getServerHostName(locator.getHost()), locatorPort);
 
     // create region in Manager, peer cache and Client cache nodes
-    manager.invoke("createRegionInManager", () -> createRegionInCache());
-    server.invoke("createRegionInPeerServer", () -> createRegionInCache());
+    manager.invoke("createRegionInManager",() -> createRegionInManager());
+    server.invoke("createRegionInPeerServer", () -> createRegionInPeerServer());
     client.invoke("createRegionInClientCache", () -> createRegionInClientCache());
 
     // do some person puts from clientcache
@@ -221,114 +260,191 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     helper.closeCache(client);
   }
 
-  private void sslPropertyConverter(Properties properties, Properties newProperties, String propertyName, String newPropertyName) {
-    String property = properties.getProperty(propertyName);
-    if (property != null) {
-      newProperties.setProperty((newPropertyName != null ? newPropertyName : propertyName), property);
+  private void createClientCacheInVM(VM vm, final String host, final int port) throws Exception {
+    SerializableRunnable connect = new SerializableRunnable("Start Cache client") {
+      public void run() {
+        // Connect using the GemFire locator and create a Caching_Proxy cache
+        ClientCache clientCache = new ClientCacheFactory().setPdxReadSerialized(true).addPoolLocator(host, port).create();
+        clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      }
+    };
+
+    if (vm == null) {
+      connect.run();
+    } else {
+      vm.invoke(connect);
     }
   }
 
   private void configureSSL(Properties props, Properties sslProperties, boolean clusterLevel) {
 
-    if (clusterLevel) {
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, DistributionConfig.CLUSTER_SSL_ENABLED_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, DistributionConfig.CLUSTER_SSL_KEYSTORE_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME,
-          DistributionConfig.CLUSTER_SSL_KEYSTORE_PASSWORD_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, DistributionConfig.CLUSTER_SSL_KEYSTORE_TYPE_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, DistributionConfig.CLUSTER_SSL_PROTOCOLS_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME,
-          DistributionConfig.CLUSTER_SSL_REQUIRE_AUTHENTICATION_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME, DistributionConfig.CLUSTER_SSL_TRUSTSTORE_NAME);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME,
-          DistributionConfig.CLUSTER_SSL_TRUSTSTORE_PASSWORD_NAME);
-    } else {
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME, null);
-      sslPropertyConverter(sslProperties, props, DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME, null);
+    if(clusterLevel){
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_ENABLED_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_KEYSTORE_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_KEYSTORE_PASSWORD_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_KEYSTORE_TYPE_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_PROTOCOLS_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_REQUIRE_AUTHENTICATION_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_TRUSTSTORE_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME) != null) {
+        props.setProperty(DistributionConfig.CLUSTER_SSL_TRUSTSTORE_PASSWORD_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME));
+      }
+
+    }else{
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME));
+      }
+      if (sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME) != null) {
+        props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME,
+            sslProperties.getProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME));
+      }
+
     }
+    
+   
   }
 
-  private void startManager(final String locators, final String[] regions, final Properties sslProperties) throws IOException {
-
+  private int startManagerInVM(VM vm, final String locators, final String[] regions, final Properties sslProperties) {
+    
     IgnoredException.addIgnoredException("java.net.BindException");
     IgnoredException.addIgnoredException("java.rmi.server.ExportException");
     IgnoredException.addIgnoredException("com.gemstone.gemfire.management.ManagementException");
-
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
-    props.setProperty("jmx-manager", "true");
-    props.setProperty("jmx-manager-start", "true");
-
-    Cache cache = null;
-    configureSSL(props, sslProperties, false);
-    while (true) {
-      try {
-        DistributedSystem ds = getSystem(props);
-        System.out.println("Creating cache with http-service-port " + props.getProperty("http-service-port", "7070")
-            + " and jmx-manager-port " + props.getProperty("jmx-manager-port", "1099"));
-        cache = CacheFactory.create(ds);
-        System.out.println("Successfully created cache.");
-        break;
-      } catch (ManagementException ex) {
-        if ((ex.getCause() instanceof BindException)
-            || (ex.getCause() != null && ex.getCause().getCause() instanceof BindException)) {
-          //close cache and disconnect
-          GemFireCacheImpl existingInstance = GemFireCacheImpl.getInstance();
-          if (existingInstance != null) {
-            existingInstance.close();
+    
+    SerializableCallable connect = new SerializableCallable("Start Manager ") {
+      public Object call() throws IOException {
+        Properties props = new Properties();
+        props.setProperty(DistributionConfig.MCAST_PORT_NAME, String.valueOf(0));
+        props.setProperty(DistributionConfig.LOCATORS_NAME, locators);
+        props.setProperty("jmx-manager", "true");
+        props.setProperty("jmx-manager-start", "true");
+
+        Cache cache = null;
+        while (true) {
+          try {
+            configureSSL(props, sslProperties, false);
+            DistributedSystem ds = getSystem(props);
+            System.out.println("Creating cache with http-service-port " + props.getProperty("http-service-port", "7070") 
+            + " and jmx-manager-port " + props.getProperty("jmx-manager-port", "1099"));            
+            cache = CacheFactory.create(ds);
+            System.out.println("Successfully created cache.");
+            break;
           }
-          InternalDistributedSystem ids = InternalDistributedSystem
-              .getConnectedInstance();
-          if (ids != null) {
-            ids.disconnect();
+          catch (ManagementException ex) {
+            if ((ex.getCause() instanceof BindException) 
+                || (ex.getCause() != null && ex.getCause().getCause() instanceof BindException)) {
+              //close cache and disconnect
+              GemFireCacheImpl existingInstance = GemFireCacheImpl.getInstance();
+              if (existingInstance != null) {
+                existingInstance.close();
+              }
+              InternalDistributedSystem ids = InternalDistributedSystem
+                  .getConnectedInstance();
+              if (ids != null) {
+                ids.disconnect();
+              }
+              //try a different port
+              int httpServicePort = AvailablePortHelper.getRandomAvailableTCPPort();
+              int jmxManagerPort = AvailablePortHelper.getRandomAvailableTCPPort();
+              props.setProperty("http-service-port", Integer.toString(httpServicePort));
+              props.setProperty("jmx-manager-port", Integer.toString(jmxManagerPort));
+              System.out.println("Try a different http-service-port " + httpServicePort);
+              System.out.println("Try a different jmx-manager-port " + jmxManagerPort);
+            }
+            else {
+              throw ex;
+            }
           }
-          //try a different port
-          int httpServicePort = AvailablePortHelper.getRandomAvailableTCPPort();
-          int jmxManagerPort = AvailablePortHelper.getRandomAvailableTCPPort();
-          props.setProperty("http-service-port", Integer.toString(httpServicePort));
-          props.setProperty("jmx-manager-port", Integer.toString(jmxManagerPort));
-          System.out.println("Try a different http-service-port " + httpServicePort);
-          System.out.println("Try a different jmx-manager-port " + jmxManagerPort);
-        } else {
-          throw ex;
+        } 
+        AttributesFactory factory = new AttributesFactory();
+
+        factory.setEnableBridgeConflation(true);
+        factory.setDataPolicy(DataPolicy.REPLICATE);
+        RegionAttributes attrs = factory.create();
+        for (int i = 0; i < regions.length; i++) {
+          cache.createRegion(regions[i], attrs);
         }
-      }
-    }
-    AttributesFactory factory = new AttributesFactory();
+        CacheServer server = cache.addCacheServer();
+        final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
+        server.setPort(serverPort);
+        server.start();
 
-    factory.setEnableBridgeConflation(true);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attrs = factory.create();
-    for (int i = 0; i < regions.length; i++) {
-      cache.createRegion(regions[i], attrs);
-    }
-    CacheServer server = cache.addCacheServer();
-    server.setPort(0);
-    server.start();
+        return new Integer(serverPort);
+      }
+    };
+    Integer port = (Integer) vm.invoke(connect);
+    return port.intValue();
   }
 
   private void createRegionInClientCache() {
-    ClientCache clientCache = ClientCacheFactory.getAnyInstance();
-    assertNotNull(clientCache);
-    clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(PEOPLE_REGION_NAME);
-    clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+    ClientCache cache = GemFireCacheImpl.getInstance();
+    assertNotNull(cache);
+    ClientRegionFactory<String, Object> crf = cache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+    crf.create(PEOPLE_REGION_NAME);
   }
 
-  private void createRegionInCache() {
+  private void createRegionInManager() {
     Cache cache = GemFireCacheImpl.getInstance();
     assertNotNull(cache);
-    RegionFactory<String, Object> regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
-    regionFactory.create(PEOPLE_REGION_NAME);
+    RegionFactory<String, Object> rf = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    rf.create(PEOPLE_REGION_NAME);
   }
 
-  private CloseableHttpClient getSSLBasedHTTPClient() throws Exception {
+  private void createRegionInPeerServer() {
+    Cache cache = GemFireCacheImpl.getInstance();
+    assertNotNull(cache);
+    RegionFactory<String, Object> rf = cache.createRegionFactory(RegionShortcut.REPLICATE);
+    rf.create(PEOPLE_REGION_NAME);
+  }
+  
+  private CloseableHttpClient getSSLBasedHTTPClient(String algo) throws Exception {
+    
     File jks = findTrustedJKS();
 
     KeyStore clientKeys = KeyStore.getInstance("JKS");
@@ -338,7 +454,7 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     SSLContext sslcontext = SSLContexts.custom()
         .loadTrustMaterial(clientKeys, new TrustSelfSignedStrategy())
         .loadKeyMaterial(clientKeys, "password".toCharArray())
-        .build();
+    .build();
 
     // Host checking is disabled here , as tests might run on multiple hosts and
     // host entries can not be assumed
@@ -350,7 +466,7 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     return httpclient;
   }
 
-  private void validateConnection(String restEndpoint) {
+  private void validateConnection(String restEndpoint, String algo) {
 
     try {
       // 1. Get on key="1" and validate result.
@@ -359,7 +475,8 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
         get.addHeader("Content-Type", "application/json");
         get.addHeader("Accept", "application/json");
 
-        CloseableHttpClient httpclient = getSSLBasedHTTPClient();
+       
+        CloseableHttpClient httpclient = getSSLBasedHTTPClient(algo);
         CloseableHttpResponse response = httpclient.execute(get);
 
         HttpEntity entity = response.getEntity();
@@ -384,7 +501,7 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
       throw new RuntimeException("unexpected exception", e);
     }
   }
-
+  
   // Actual Tests starts here.
 
   public void testSimpleSSL() throws Exception {
@@ -394,80 +511,90 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_TYPE_NAME, "JKS");
-    validateConnection(startInfraWithSSL(props, false));
+    String restEndpoint = startInfraWithSSL(props,false);
+    validateConnection(restEndpoint, "SSL");
   }
-
+  
   public void testSSLWithoutKeyStoreType() throws Exception {
 
+   
+
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
+  
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "SSL");
 
-    validateConnection(startInfraWithSSL(props, false));
 
   }
-
+  
   public void testSSLWithSSLProtocol() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL");
-
-    validateConnection(startInfraWithSSL(props, false));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL");
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "SSL");
 
   }
-
+  
   public void testSSLWithTLSProtocol() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLS");
-
-    validateConnection(startInfraWithSSL(props, false));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLS");
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "TLS");
 
   }
-
+  
   public void testSSLWithTLSv11Protocol() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.1");
-
-    validateConnection(startInfraWithSSL(props, false));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.1");
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "TLSv1.1");
 
   }
-
+  
   public void testSSLWithTLSv12Protocol() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
-
-    validateConnection(startInfraWithSSL(props, false));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "TLSv1.2");
 
   }
-
+  
   public void testWithMultipleProtocol() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL,TLSv1.2");
-
-    validateConnection(startInfraWithSSL(props, false));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL,TLSv1.2");
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "TLSv1.2");
 
   }
-
+  
   public void testSSLWithCipherSuite() throws Exception {
 
     System.setProperty("javax.net.debug", "ssl");
@@ -475,51 +602,57 @@ public class RestAPIsWithSSLDUnitTest extends LocatorTestBase {
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
-
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    
     SSLContext ssl = SSLContext.getInstance("TLSv1.2");
-
+    
     ssl.init(null, null, new java.security.SecureRandom());
     String[] cipherSuites = ssl.getSocketFactory().getSupportedCipherSuites();
-
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_CIPHERS_NAME, cipherSuites[0]);
-
-    validateConnection(startInfraWithSSL(props, false));
+    
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_CIPHERS_NAME,cipherSuites[0]);
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "TLSv1.2");
 
   }
-
+  
   public void testSSLWithMultipleCipherSuite() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "TLSv1.2");
-
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"TLSv1.2");
+    
     SSLContext ssl = SSLContext.getInstance("TLSv1.2");
-
+    
     ssl.init(null, null, new java.security.SecureRandom());
     String[] cipherSuites = ssl.getSocketFactory().getSupportedCipherSuites();
-
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_CIPHERS_NAME, cipherSuites[0] + "," + cipherSuites[1]);
-
-    validateConnection(startInfraWithSSL(props, false));
+    
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_CIPHERS_NAME,cipherSuites[0]+","+cipherSuites[1]);
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "TLSv1.2");
 
   }
-
+  
+  
   public void testMutualAuthentication() throws Exception {
 
     Properties props = new Properties();
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_NAME, jks.getCanonicalPath());
     props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_KEYSTORE_PASSWORD_NAME, "password");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME, "SSL");
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME, "true");
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_PROTOCOLS_NAME,"SSL");
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_REQUIRE_AUTHENTICATION_NAME,"true");
 
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME, jks.getCanonicalPath());
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_NAME,jks.getCanonicalPath());
 
-    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME, "password");
-    validateConnection(startInfraWithSSL(props, false));
+    props.setProperty(DistributionConfig.HTTP_SERVICE_SSL_TRUSTSTORE_PASSWORD_NAME,"password");
+    
+    String restEndpoint = startInfraWithSSL(props, false);
+    validateConnection(restEndpoint, "SSL");
   }
 
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
index ad578c5..cfcff5e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
@@ -16,6 +16,15 @@
  */
 package com.gemstone.gemfire.cache.client.internal;
 
+import java.io.Serializable;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+
+import junit.framework.Assert;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.NoAvailableLocatorsException;
@@ -28,23 +37,22 @@ import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.management.membership.ClientMembership;
 import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
 import com.gemstone.gemfire.management.membership.ClientMembershipListenerAdapter;
-import com.gemstone.gemfire.test.dunit.*;
-import org.junit.Assert;
-
-import java.io.Serializable;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
 
 /**
  * Tests cases that are particular for the auto connection source
- * - dynamically discovering servers, locators, handling
+ * - dynamically discovering servers, locators, handling 
  * locator disappearance, etc.
+ *
  */
 public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
-
+  
   protected static final Object BRIDGE_LISTENER = "BRIDGE_LISTENER";
   private static final long MAX_WAIT = 60000;
 
@@ -56,85 +64,84 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
   public AutoConnectionSourceDUnitTest(String name) {
     super(name);
   }
-
+  
   public void testDiscoverBridgeServers() 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);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
-
-    String locators = NetworkUtils.getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
+    startLocatorInVM(vm0, locatorPort, "");
+    
+    String locators = NetworkUtils.getServerHostName(vm0.getHost())+ "[" + locatorPort + "]";
+    
+    startBridgeServerInVM(vm1, null, locators);
 
-    vm1.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
-    vm2.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort));
+    startBridgeClientInVM(vm2, null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort);
 
     putAndWaitForSuccess(vm2, REGION_NAME, "key", "value");
-
+    
     Assert.assertEquals("value", getInVM(vm1, "key"));
   }
 
   public void testNoLocators() {
-
+    
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
-
+    
     try {
-      vm0.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(vm0.getHost())
-          , AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET)));
+      startBridgeClientInVM(vm0, null, NetworkUtils.getServerHostName(vm0.getHost()), AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET));
       putInVM(vm0, "key", "value");
       fail("Client cache should not have been able to start");
-    } catch (Exception e) {
+    } catch(Exception e) {
       //expected an exception
     }
   }
-
+  
   public void testNoBridgeServer() {
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
-    try {
-      vm1.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort));
+    startLocatorInVM(vm0, locatorPort, "");
+    try { 
+      startBridgeClientInVM(vm1, null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort);
       putInVM(vm0, "key", "value");
       fail("Client cache should not have been able to start");
-    } catch (Exception e) {
+    } catch(Exception e) {
       //expected an exception
     }
   }
-
-  public void testDynamicallyFindBridgeServer() throws Exception {
+  
+  public void testDynamicallyFindBridgeServer() throws Exception  {
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
-
+    startLocatorInVM(vm0, locatorPort, "");
+    
     String locators = NetworkUtils.getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
-
-    vm1.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
-    vm2.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort));
-
+    
+    startBridgeServerInVM(vm1, null, locators);
+    
+    startBridgeClientInVM(vm2, null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort);
+    
     putAndWaitForSuccess(vm2, REGION_NAME, "key", "value");
-
-    vm3.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
+    
+    startBridgeServerInVM(vm3, null, locators);
+    
     stopBridgeMemberVM(vm1);
-
+    
     putAndWaitForSuccess(vm2, REGION_NAME, "key2", "value2");
-
+    
     Assert.assertEquals("value2", getInVM(vm3, "key2"));
   }
-
+  
   public void testDynamicallyFindLocators() throws Exception {
     try {
       final Host host = Host.getHost(0);
@@ -143,60 +150,59 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       VM vm1 = host.getVM(1);
       VM vm2 = host.getVM(2);
       VM vm3 = host.getVM(3);
-
+      
       int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
-
+      
       final int locatorPort0 = ports[0];
       final int locatorPort1 = ports[1];
       final int locatorPort3 = ports[2];
-      String locators = getLocatorString(host, new int[] { locatorPort0, locatorPort1, locatorPort3 });
-      vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort0, locators));
-      vm1.invoke("Start Locator", () -> startLocator(vm1.getHost(), locatorPort1, locators));
-
-      vm2.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort0));
-
-      InetSocketAddress locatorToWaitFor = new InetSocketAddress(hostName, locatorPort1);
+      String locators = getLocatorString(host, new int[] { locatorPort0, locatorPort1, locatorPort3});
+      startLocatorInVM(vm0, locatorPort0, locators);
+      
+      startLocatorInVM(vm1, locatorPort1, locators);
+      startBridgeClientInVM(vm2, null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort0);
+      
+      InetSocketAddress locatorToWaitFor= new InetSocketAddress(hostName, locatorPort1);
       waitForLocatorDiscovery(vm2, locatorToWaitFor);
-
-      vm0.invoke("Stop Locator", () -> stopLocator());
-      vm0.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
+      
+      stopLocatorInVM(vm0);
+      startBridgeServerInVM(vm0, null, locators);
+      
       putAndWaitForSuccess(vm2, REGION_NAME, "key", "value");
       Assert.assertEquals("value", getInVM(vm0, "key"));
-
-      vm3.invoke("Start Locator", () -> startLocator(vm3.getHost(), locatorPort3, locators));
+      
+      startLocatorInVM(vm3, locatorPort3, locators);
       stopBridgeMemberVM(vm0);
-      locatorToWaitFor = new InetSocketAddress(hostName, locatorPort3);
+      locatorToWaitFor= new InetSocketAddress(hostName, locatorPort3);
       waitForLocatorDiscovery(vm2, locatorToWaitFor);
-      vm1.invoke("Stop Locator", () -> stopLocator());
-      vm1.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
+      stopLocatorInVM(vm1);
+      startBridgeServerInVM(vm1, null, locators);
       putAndWaitForSuccess(vm2, REGION_NAME, "key2", "value2");
       Assert.assertEquals("value2", getInVM(vm1, "key2"));
-    } catch (Exception ec) {
-      if (ec.getCause() != null && (ec.getCause().getCause() instanceof BindException))
+    }catch(Exception ec) {
+      if(ec.getCause() != null && (ec.getCause().getCause() instanceof BindException))
         return;//BindException let it pass
       throw ec;
     }
   }
-
-  public void testEmbeddedLocator() throws Exception {
+  
+  public void testEmbeddedLocator() throws Exception  {
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
+    
     String locators = NetworkUtils.getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
-
-    vm0.invoke("Start BridgeServer", () -> startBridgeServerWithEmbeddedLocator(null, locators, new String[] { REGION_NAME }
-        , CacheServer.DEFAULT_LOAD_PROBE));
-
-    vm2.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort));
-
+    
+    startBridgeServerWithEmbeddedLocator(vm0, null, locators, new String[] {REGION_NAME}, CacheServer.DEFAULT_LOAD_PROBE);
+    
+    startBridgeClientInVM(vm2, null, NetworkUtils.getServerHostName(vm0.getHost()), locatorPort);
+    
     putAndWaitForSuccess(vm2, REGION_NAME, "key", "value");
-
+    
     Assert.assertEquals("value", getInVM(vm2, "key"));
   }
 
@@ -205,7 +211,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     vm.invoke(new SerializableCallable() {
       public Object call() throws InterruptedException {
         MyLocatorCallback callback = (MyLocatorCallback) remoteObjects.get(CALLBACK_KEY);
-
+        
         boolean discovered = callback.waitForDiscovery(locatorToWaitFor, MAX_WAIT);
         Assert.assertTrue("Waited " + MAX_WAIT + " for " + locatorToWaitFor
             + " to be discovered on client. List is now: "
@@ -214,89 +220,85 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   public void testServerGroups() throws Exception {
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
     VM vm3 = host.getVM(3);
-
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
-
+    startLocatorInVM(vm0, locatorPort, "");
+    
     String locators = NetworkUtils.getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
+    
+    startBridgeServerInVM(vm1, new String[] {"group1", "group2"} , locators, new String[] {"A", "B"});
+    startBridgeServerInVM(vm2, new String[] {"group2", "group3"}, locators, new String[] {"B", "C"});
 
-    vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "group1", "group2" }, locators, new String[] { "A", "B" }));
-    vm2.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "group2", "group3" }, locators, new String[] { "B", "C" }));
-
-    vm3.invoke("StartBridgeClient", () -> startBridgeClient("group1", NetworkUtils.getServerHostName(vm0.getHost())
-        , locatorPort, new String[] { "A", "B", "C" }));
+    
+    startBridgeClientInVM(vm3, "group1", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort, new String [] {"A", "B", "C"});
     putAndWaitForSuccess(vm3, "A", "key", "value");
     Assert.assertEquals("value", getInVM(vm1, "A", "key"));
     try {
       putInVM(vm3, "C", "key2", "value2");
       fail("Should not have been able to find Region C on the server");
-    } catch (Exception expected) {
-    }
-
+    } catch(Exception expected) {}
+    
     stopBridgeMemberVM(vm3);
-
-    vm3.invoke("StartBridgeClient", () -> startBridgeClient("group3", NetworkUtils.getServerHostName(vm0.getHost()),
-        locatorPort, new String[] { "A", "B", "C" }));
+    
+    startBridgeClientInVM(vm3, "group3", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort, new String [] {"A", "B", "C"});
     try {
       putInVM(vm3, "A", "key3", "value");
       fail("Should not have been able to find Region A on the server");
-    } catch (Exception expected) {
-    }
+    } catch(Exception expected) {}
     putInVM(vm3, "C", "key4", "value");
     Assert.assertEquals("value", getInVM(vm2, "C", "key4"));
-
+    
     stopBridgeMemberVM(vm3);
-
-    vm3.invoke("StartBridgeClient", () -> startBridgeClient("group2", NetworkUtils.getServerHostName(vm0.getHost()),
-        locatorPort, new String[] { "A", "B", "C" }));
+    
+    startBridgeClientInVM(vm3, "group2", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort, new String [] {"A", "B", "C"});
     putInVM(vm3, "B", "key5", "value");
     Assert.assertEquals("value", getInVM(vm1, "B", "key5"));
     Assert.assertEquals("value", getInVM(vm2, "B", "key5"));
-
+    
     stopBridgeMemberVM(vm1);
     putInVM(vm3, "B", "key6", "value");
     Assert.assertEquals("value", getInVM(vm2, "B", "key6"));
-    vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "group1", "group2" }, locators, new String[] { "A", "B" }));
+    startBridgeServerInVM(vm1, new String[] {"group1", "group2"} , locators, new String[] {"A", "B"});
     stopBridgeMemberVM(vm2);
-
+    
     putInVM(vm3, "B", "key7", "value");
     Assert.assertEquals("value", getInVM(vm1, "B", "key7"));
   }
-
+  
   public void testTwoServersInSameVM() throws Exception {
     final Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
     VM vm2 = host.getVM(2);
-    //    VM vm3 = host.getVM(3);
-
+//    VM vm3 = host.getVM(3);
+    
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-
-    vm0.invoke("Start Locator", () -> startLocator(vm0.getHost(), locatorPort, ""));
-
+    
+    startLocatorInVM(vm0, locatorPort, "");
+    
     final String locators = NetworkUtils.getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
-
-    final int serverPort1 = vm1.invoke("Start BridgeServer", () -> startBridgeServer(new String[] { "group1" }, locators));
-    final int serverPort2 = vm1.invoke("Start CacheServer", () -> addCacheServer(new String[] { "group2" }));
-
-    vm2.invoke("StartBridgeClient", () -> startBridgeClient("group2", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort));
-
-    checkEndpoints(vm2, new int[] { serverPort2 });
-
+    
+    final int serverPort1 =startBridgeServerInVM(vm1, new String[] {"group1"}, locators);
+    final int serverPort2 =addCacheServerInVM(vm1, new String[] {"group2"});
+    
+    startBridgeClientInVM(vm2, "group2", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort);
+    
+    checkEndpoints(vm2, new int[] {serverPort2});
+    
     stopBridgeMemberVM(vm2);
 
-    vm2.invoke("StartBridgeClient", () -> startBridgeClient("group1", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort));
-
-    checkEndpoints(vm2, new int[] { serverPort1 });
+    startBridgeClientInVM(vm2, "group1", NetworkUtils.getServerHostName(vm0.getHost()), locatorPort);
+    
+    checkEndpoints(vm2, new int[] {serverPort1});
   }
-
+  
   public void testClientMembershipListener() throws Exception {
     final Host host = Host.getHost(0);
     VM locatorVM = host.getVM(0);
@@ -304,20 +306,19 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     VM bridge2VM = host.getVM(2);
     VM clientVM = host.getVM(3);
     int locatorPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    locatorVM.invoke("Start Locator", () -> startLocator(locatorVM.getHost(), locatorPort, ""));
-
+    startLocatorInVM(locatorVM, locatorPort, "");
     String locators = NetworkUtils.getServerHostName(locatorVM.getHost()) + "[" + locatorPort + "]";
 
     //start a bridge server with a listener
     addBridgeListener(bridge1VM);
-    int serverPort1 = bridge1VM.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
+    int serverPort1 = startBridgeServerInVM(bridge1VM, null, locators);
 
     //start a bridge client with a listener
     addBridgeListener(clientVM);
-    clientVM.invoke("StartBridgeClient", () -> startBridgeClient(null, NetworkUtils.getServerHostName(locatorVM.getHost()), locatorPort));
+    startBridgeClientInVM(clientVM, null, NetworkUtils.getServerHostName(locatorVM.getHost()), locatorPort);
     // wait for client to connect
-    checkEndpoints(clientVM, new int[] { serverPort1 });
-
+    checkEndpoints(clientVM, new int[] {serverPort1});
+    
     //make sure the client and bridge server both noticed each other
     waitForJoin(bridge1VM);
     MyListener serverListener = getBridgeListener(bridge1VM);
@@ -325,48 +326,48 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     Assert.assertEquals(0, serverListener.getDepartures());
     Assert.assertEquals(1, serverListener.getJoins());
     resetBridgeListener(bridge1VM);
-
+    
     waitForJoin(clientVM);
-    MyListener clientListener = getBridgeListener(clientVM);
+    MyListener clientListener= getBridgeListener(clientVM);
     Assert.assertEquals(0, clientListener.getCrashes());
     Assert.assertEquals(0, clientListener.getDepartures());
     Assert.assertEquals(1, clientListener.getJoins());
     resetBridgeListener(clientVM);
-
-    checkEndpoints(clientVM, new int[] { serverPort1 });
+    
+    checkEndpoints(clientVM, new int[] {serverPort1});
 
     //start another bridge server and make sure it is detected by the client
-    int serverPort2 = bridge2VM.invoke("Start BridgeServer", () -> startBridgeServer(null, locators));
-
-    checkEndpoints(clientVM, new int[] { serverPort1, serverPort2 });
+    int serverPort2 = startBridgeServerInVM(bridge2VM, null, locators);
+    
+    checkEndpoints(clientVM, new int[] {serverPort1, serverPort2});
     serverListener = getBridgeListener(bridge1VM);
     Assert.assertEquals(0, serverListener.getCrashes());
     Assert.assertEquals(0, serverListener.getDepartures());
     Assert.assertEquals(0, serverListener.getJoins());
     resetBridgeListener(bridge1VM);
     waitForJoin(clientVM);
-    clientListener = getBridgeListener(clientVM);
+    clientListener= getBridgeListener(clientVM);
     Assert.assertEquals(0, clientListener.getCrashes());
     Assert.assertEquals(0, clientListener.getDepartures());
     Assert.assertEquals(1, clientListener.getJoins());
     resetBridgeListener(clientVM);
-
+    
     //stop the second bridge server and make sure it is detected by the client
     stopBridgeMemberVM(bridge2VM);
-
-    checkEndpoints(clientVM, new int[] { serverPort1 });
+    
+    checkEndpoints(clientVM, new int[] {serverPort1});
     serverListener = getBridgeListener(bridge1VM);
     Assert.assertEquals(0, serverListener.getCrashes());
     Assert.assertEquals(0, serverListener.getDepartures());
     Assert.assertEquals(0, serverListener.getJoins());
     resetBridgeListener(bridge1VM);
     waitForCrash(clientVM);
-    clientListener = getBridgeListener(clientVM);
+    clientListener= getBridgeListener(clientVM);
     Assert.assertEquals(1, clientListener.getCrashes());
     Assert.assertEquals(0, clientListener.getDepartures());
     Assert.assertEquals(0, clientListener.getJoins());
     resetBridgeListener(clientVM);
-
+    
     //stop the client and make sure the bridge server notices
     stopBridgeMemberVM(clientVM);
     waitForDeparture(bridge1VM);
@@ -379,7 +380,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
   protected Object getInVM(VM vm, final Serializable key) {
     return getInVM(vm, REGION_NAME, key);
   }
-
+  
   protected Object getInVM(VM vm, final String regionName, final Serializable key) {
     return vm.invoke(new SerializableCallable("Get in VM") {
       public Object call() throws Exception {
@@ -389,22 +390,23 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
-  protected void putAndWaitForSuccess(VM vm, final String regionName, final Serializable key, final Serializable value) throws InterruptedException {
+  
+  protected void putAndWaitForSuccess(VM vm,  final String regionName, final Serializable key, final Serializable value) throws InterruptedException
+  {
     long endTime = System.currentTimeMillis() + MAX_WAIT;
     long remaining = MAX_WAIT;
     int i = 0;
-    while (true) {
+    while(true) {
       try {
         System.err.println("Attempt: " + (i++));
         putInVM(vm, regionName, key, value);
         break;
       } catch (NoAvailableLocatorsException | com.gemstone.gemfire.test.dunit.RMIException e) {
-        if (!(e instanceof NoAvailableLocatorsException)
+        if( !(e instanceof NoAvailableLocatorsException)
             && !(e.getCause() instanceof NoAvailableServersException)) {
           throw e;
         }
-        if (remaining <= 0) {
+        if(remaining <= 0) {
           throw e;
         }
         Wait.pause(100);
@@ -416,8 +418,10 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
   protected void putInVM(VM vm, final Serializable key, final Serializable value) {
     putInVM(vm, REGION_NAME, key, value);
   }
-
-  protected void putInVM(VM vm, final String regionName, final Serializable key, final Serializable value) {
+  
+  
+  
+  protected void putInVM(VM vm,  final String regionName, final Serializable key, final Serializable value) {
     vm.invoke(new SerializableCallable("Put in VM") {
       public Object call() throws Exception {
         Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
@@ -426,12 +430,11 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   /**
    * Assert that there is one endpoint with the given host in port
    * on the client vm.
-   *
-   * @param vm            - the vm the client is running in
+   * @param vm - the vm the client is running in
    * @param expectedPorts - The server ports we expect the client to be connected to.
    */
   protected void checkEndpoints(VM vm, final int[] expectedPorts) {
@@ -442,14 +445,14 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
         List/*<ServerLocation>*/ endpoints;
         HashSet actualEndpointPorts;
         HashSet expectedEndpointPorts = new HashSet();
-        for (int i = 0; i < expectedPorts.length; i++) {
+        for(int i = 0; i < expectedPorts.length; i++) {
           expectedEndpointPorts.add(new Integer(expectedPorts[i]));
         }
         do {
           endpoints = pool.getCurrentServers();
           actualEndpointPorts = new HashSet();
-          for (Iterator itr = endpoints.iterator(); itr.hasNext(); ) {
-            ServerLocation sl = (ServerLocation) itr.next();
+          for(Iterator itr = endpoints.iterator(); itr.hasNext();) {
+            ServerLocation sl = (ServerLocation)itr.next();
             actualEndpointPorts.add(new Integer(sl.getPort()));
           }
           if (expectedEndpointPorts.size() == actualEndpointPorts.size()) {
@@ -457,12 +460,12 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
           } else {
             Wait.pause(100);
           }
-        } while (retryCount-- > 0);
+        } while(retryCount-- > 0);
         Assert.assertEquals(expectedEndpointPorts, actualEndpointPorts);
       }
     });
   }
-
+  
   protected void addBridgeListener(VM vm) {
     vm.invoke(new SerializableRunnable("Add membership listener") {
       public void run() {
@@ -472,7 +475,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   protected void resetBridgeListener(VM vm) {
     vm.invoke(new SerializableRunnable("Add membership listener") {
       public void run() {
@@ -481,7 +484,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   private MyListener getBridgeListener(VM vm) {
     return (MyListener) vm.invoke(new SerializableCallable("Add membership listener") {
       public Object call() {
@@ -489,17 +492,17 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   private void waitForJoin(VM vm) {
     vm.invoke(new SerializableRunnable() {
       public void run() {
         MyListener listener = (MyListener) remoteObjects.get(BRIDGE_LISTENER);
-        synchronized (listener) {
+        synchronized(listener) {
           long end = System.currentTimeMillis() + 10000;
           while (listener.joins == 0) {
             try {
               long remaining = end - System.currentTimeMillis();
-              if (remaining < 0) {
+              if(remaining < 0) {
                 break;
               }
               listener.wait(remaining);
@@ -511,17 +514,17 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   private void waitForCrash(VM vm) {
     vm.invoke(new SerializableRunnable() {
       public void run() {
         MyListener listener = (MyListener) remoteObjects.get(BRIDGE_LISTENER);
-        synchronized (listener) {
+        synchronized(listener) {
           long end = System.currentTimeMillis() + 10000;
-          while (listener.crashes == 0) {
+          while (listener.crashes== 0) {
             try {
               long remaining = end - System.currentTimeMillis();
-              if (remaining < 0) {
+              if(remaining < 0) {
                 break;
               }
               listener.wait(remaining);
@@ -533,17 +536,17 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   private void waitForDeparture(VM vm) {
     vm.invoke(new SerializableRunnable() {
       public void run() {
         MyListener listener = (MyListener) remoteObjects.get(BRIDGE_LISTENER);
-        synchronized (listener) {
+        synchronized(listener) {
           long end = System.currentTimeMillis() + 10000;
           while (listener.departures == 0) {
             try {
               long remaining = end - System.currentTimeMillis();
-              if (remaining < 0) {
+              if(remaining < 0) {
                 break;
               }
               listener.wait(remaining);
@@ -555,11 +558,11 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       }
     });
   }
-
+  
   public static class MyListener extends ClientMembershipListenerAdapter implements Serializable {
     protected int crashes = 0;
     protected int joins = 0;
-    protected int departures = 0;
+    protected int departures= 0;
 
     @Override
     public synchronized void memberCrashed(ClientMembershipEvent event) {
@@ -593,7 +596,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       return joins;
     }
 
-    public synchronized int getDepartures() {
+    public synchronized int  getDepartures() {
       return departures;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1a0748b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
index 667af89..8deedc1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceImplJUnitTest.java
@@ -144,22 +144,27 @@ public class AutoConnectionSourceImplJUnitTest {
   
   @Test
   public void testNoServers() throws Exception {
+    
     startFakeLocator();
     handler.nextConnectionResponse = new ClientConnectionResponse(null);
+    
     assertEquals(null, source.findServer(null));
   }
   
   @Test
   public void testDiscoverServers() throws Exception {
     startFakeLocator();
+    
     ServerLocation loc1 = new ServerLocation("localhost", 4423);
     handler.nextConnectionResponse = new ClientConnectionResponse(loc1);
+    
     assertEquals(loc1, source.findServer(null));
   }
   
   @Test
   public void testDiscoverLocators() throws Exception {
     startFakeLocator();
+    
     int secondPort = AvailablePortHelper.getRandomAvailableTCPPort();
     TcpServer server2 = new TcpServer(secondPort, InetAddress.getLocalHost(), null, null, handler, new FakeHelper(), Thread.currentThread().getThreadGroup(), "tcp server");
     server2.start();


[18/50] [abbrv] incubator-geode git commit: GEODE-17: make geode authorization case-sensitive since our region names are case sensitive

Posted by kl...@apache.org.
GEODE-17: make geode authorization case-sensitive since our region names are case sensitive

* Specify case sensitive when creating the permission context
* Specify case sensitive when resolving the permission from shiro-ini file
* rename shiro-init to security-shiro-init since it's security related in DistributionConfig
* For DATA operations, a "NULL" regionName is used when regionName couldn't be resolved yet. Since for permissions,
  DATA:READ is different from DATA:READ:NULL


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

Branch: refs/heads/feature/GEODE-835
Commit: 84a9eb1b4c43a1109ac306f7743fc5af0d37827c
Parents: 3e8a610
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri May 13 14:04:10 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue May 17 08:29:30 2016 -0700

----------------------------------------------------------------------
 .../cache/operations/OperationContext.java      | 10 ++++-
 .../internal/AbstractDistributionConfig.java    |  2 +-
 .../internal/DistributionConfig.java            | 26 ++++++++-----
 .../gemfire/internal/AbstractConfig.java        |  2 +-
 .../internal/security/GeodeSecurityUtil.java    | 41 ++++++++++++++++++++
 .../security/shiro/GeodePermissionResolver.java | 28 +++++++++++++
 .../internal/SystemManagementService.java       | 29 ++------------
 .../security/ResourceOperationContext.java      | 10 ++++-
 .../CacheServerMBeanShiroJUnitTest.java         |  2 +-
 .../security/DataCommandsSecurityTest.java      |  4 +-
 .../GeodeSecurityUtilCustomRealmJUnitTest.java  | 18 ++-------
 .../GeodeSecurityUtilWithIniFileJUnitTest.java  | 15 ++++---
 .../security/GfshCommandsSecurityTest.java      |  2 +-
 .../ResourceOperationContextJUnitTest.java      | 11 ++++--
 .../internal/security/ShiroCacheStartRule.java  |  2 +-
 .../internal/security/TestCommand.java          | 16 ++++----
 16 files changed, 140 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
index dec716c..b81016d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/operations/OperationContext.java
@@ -304,7 +304,7 @@ public abstract class OperationContext extends WildcardPermission{
   }
 
   public String getRegionName(){
-    return "NULL";
+    return null;
   }
 
   /**
@@ -358,4 +358,12 @@ public abstract class OperationContext extends WildcardPermission{
         || opCode.isRegionDestroy() || opCode.isRegionClear());
   }
 
+  @Override
+  public String toString(){
+    if(getRegionName()==null)
+      return getResource()+":"+getOperationCode();
+    else
+      return getResource()+":"+getOperationCode()+":"+getRegionName();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
index d38e1a9..17e7c2b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
@@ -1125,7 +1125,7 @@ public abstract class AbstractDistributionConfig
     m.put(LOCK_MEMORY_NAME, LocalizedStrings.AbstractDistributionConfig_LOCK_MEMORY.toLocalizedString(DEFAULT_LOCK_MEMORY));
     m.put(DISTRIBUTED_TRANSACTIONS_NAME, "Flag to indicate whether all transactions including JTA should be distributed transactions.  Default is false, meaning colocated transactions.");
 
-    m.put(SHIRO_INIT_NAME, "The name of the shiro configuration file in the classpath, e.g. shiro.ini");
+    m.put(SECURITY_SHIRO_INIT_NAME, "The name of the shiro configuration file in the classpath, e.g. shiro.ini");
 
     dcAttDescriptions = Collections.unmodifiableMap(m);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
index c0e560c..36ef671 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfig.java
@@ -17,6 +17,17 @@
 
 package com.gemstone.gemfire.distributed.internal;
 
+import java.io.File;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.Config;
 import com.gemstone.gemfire.internal.ConfigSource;
@@ -25,12 +36,6 @@ import com.gemstone.gemfire.internal.logging.LogConfig;
 import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.memcached.GemFireMemcachedServer;
 
-import java.io.File;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.net.InetAddress;
-import java.util.*;
-
 /**
  * Provides accessor (and in some cases mutator) methods for the
  * various GemFire distribution configuration properties.  The
@@ -47,7 +52,8 @@ import java.util.*;
  *
  * @since 2.1
  */
-public interface DistributionConfig extends Config, LogConfig {
+public interface
+DistributionConfig extends Config, LogConfig {
 
   ////////////////////  Instance Methods  ////////////////////
 
@@ -3739,11 +3745,11 @@ public interface DistributionConfig extends Config, LogConfig {
   public void setLockMemory(boolean value);
 
   @ConfigAttribute(type=String.class)
-  public String SHIRO_INIT_NAME="shiro-init";
+  public String SECURITY_SHIRO_INIT_NAME ="security-shiro-init";
 
-  @ConfigAttributeSetter(name=SHIRO_INIT_NAME)
+  @ConfigAttributeSetter(name= SECURITY_SHIRO_INIT_NAME)
   public void setShiroInit(String value);
-  @ConfigAttributeGetter(name=SHIRO_INIT_NAME)
+  @ConfigAttributeGetter(name= SECURITY_SHIRO_INIT_NAME)
   public String getShiroInit();
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
index a4c2f2f..93cb9b2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/AbstractConfig.java
@@ -172,7 +172,7 @@ public abstract class AbstractConfig implements Config {
         }
       }
       // hide the shiro-init configuration for now. Remove after we can allow customer to specify shiro.ini file
-      if(attName.equals("shiro-init")){
+      if(attName.equals(DistributionConfig.SECURITY_SHIRO_INIT_NAME)){
         continue;
       }
       pw.print(attName);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 6e10f3f..236b00b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -18,13 +18,16 @@
 package com.gemstone.gemfire.internal.security;
 
 import java.security.AccessController;
+import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
 import com.gemstone.gemfire.cache.operations.OperationContext;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.cache.operations.OperationContext.Resource;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
 import com.gemstone.gemfire.internal.security.shiro.ShiroPrincipal;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 import com.gemstone.gemfire.management.internal.security.ResourceOperationContext;
@@ -37,6 +40,11 @@ import org.apache.shiro.SecurityUtils;
 import org.apache.shiro.ShiroException;
 import org.apache.shiro.UnavailableSecurityManagerException;
 import org.apache.shiro.authc.UsernamePasswordToken;
+import org.apache.shiro.config.Ini.Section;
+import org.apache.shiro.config.IniSecurityManagerFactory;
+import org.apache.shiro.mgt.DefaultSecurityManager;
+import org.apache.shiro.mgt.SecurityManager;
+import org.apache.shiro.realm.Realm;
 import org.apache.shiro.subject.Subject;
 import org.apache.shiro.util.ThreadContext;
 
@@ -211,4 +219,37 @@ public class GeodeSecurityUtil {
     return true;
   }
 
+  /**
+   * initialize Shiro's Security Manager and Security Utilities
+   * @param securityProps
+   */
+  public static void initSecurity(Properties securityProps){
+    if(securityProps==null)
+      return;
+
+    String shiroConfig = securityProps.getProperty(DistributionConfig.SECURITY_SHIRO_INIT_NAME);
+    String customAuthenticator =securityProps.getProperty(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME);
+    if (!com.gemstone.gemfire.internal.lang.StringUtils.isBlank(shiroConfig)) {
+      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
+
+      // we will need to make sure that shiro uses a case sensitive permission resolver
+      Section main = factory.getIni().addSection("main");
+      main.put("geodePermissionResolver", "com.gemstone.gemfire.internal.security.shiro.GeodePermissionResolver");
+      if(!main.containsKey("iniRealm.permissionResolver")) {
+        main.put("iniRealm.permissionResolver", "$geodePermissionResolver");
+      }
+
+      SecurityManager securityManager = factory.getInstance();
+      SecurityUtils.setSecurityManager(securityManager);
+    }
+    else if (!com.gemstone.gemfire.internal.lang.StringUtils.isBlank(customAuthenticator)) {
+      Realm realm = new CustomAuthRealm(securityProps);
+      SecurityManager securityManager = new DefaultSecurityManager(realm);
+      SecurityUtils.setSecurityManager(securityManager);
+    }
+    else{
+      SecurityUtils.setSecurityManager(null);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
new file mode 100644
index 0000000..d170756
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/shiro/GeodePermissionResolver.java
@@ -0,0 +1,28 @@
+/*
+ * 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.security.shiro;
+
+import org.apache.shiro.authz.Permission;
+import org.apache.shiro.authz.permission.PermissionResolver;
+import org.apache.shiro.authz.permission.WildcardPermission;
+
+public class GeodePermissionResolver implements PermissionResolver {
+  @Override public Permission resolvePermission(final String permissionString) {
+    return new WildcardPermission(permissionString, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
index fd2a834..b773b94 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
@@ -28,14 +28,13 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ResourceEvent;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.AlreadyRunningException;
 import com.gemstone.gemfire.management.AsyncEventQueueMXBean;
 import com.gemstone.gemfire.management.CacheServerMXBean;
@@ -54,13 +53,8 @@ import com.gemstone.gemfire.management.RegionMXBean;
 import com.gemstone.gemfire.management.internal.beans.ManagementAdapter;
 import com.gemstone.gemfire.management.membership.MembershipEvent;
 import com.gemstone.gemfire.management.membership.MembershipListener;
-import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
+
 import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.config.IniSecurityManagerFactory;
-import org.apache.shiro.mgt.DefaultSecurityManager;
-import org.apache.shiro.mgt.SecurityManager;
-import org.apache.shiro.realm.Realm;
 import org.apache.shiro.util.ThreadContext;
 
 /**
@@ -157,24 +151,7 @@ public final class SystemManagementService extends BaseManagementService {
     this.jmxAdapter = new MBeanJMXAdapter();      
     this.repo = new ManagementResourceRepo();
 
-    DistributionConfig config = system.getConfig();
-
-    // setup shiro for authentication and authorization if it's desired
-    String shiroConfig = config.getShiroInit();
-    String customAuthenticator = config.getSecurityClientAuthenticator();
-    if (!StringUtils.isBlank(shiroConfig)) {
-      IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:"+shiroConfig);
-      SecurityManager securityManager = factory.getInstance();
-      SecurityUtils.setSecurityManager(securityManager);
-    }
-    else if (!StringUtils.isBlank(customAuthenticator)) {
-      Realm realm = new CustomAuthRealm(config.getSecurityProps());
-      SecurityManager securityManager = new DefaultSecurityManager(realm);
-      SecurityUtils.setSecurityManager(securityManager);
-    }
-    else{
-      SecurityUtils.setSecurityManager(null);
-    }
+    GeodeSecurityUtil.initSecurity(system.getConfig().getSecurityProps());
 
     this.notificationHub = new NotificationHub(repo);
     if (system.getConfig().getJmxManager()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
index 2e46104..ab49270 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContext.java
@@ -43,7 +43,15 @@ public class ResourceOperationContext extends OperationContext {
     if (operation != null) this.operation = OperationCode.valueOf(operation);
     if (regionName !=null ) this.regionName = regionName;
 
-    setParts(this.resource.name()+":"+this.operation.name()+":"+this.regionName);
+    //for DATA resource, when we construct the lock to guard the operations, there should always be a 3rd part (regionName),
+    // if no regionName is specified, we need to add "NULL" to it.
+    // this means, for general data operations, or operations that we can't put a regionName on yet, like backup diskstore, query data, create regions
+    // it will require DATA:REAT/WRITE:NULL role
+    if(this.resource==Resource.DATA && this.regionName==null){
+      this.regionName = "NULL";
+    }
+
+    setParts(this.resource.name()+":"+this.operation.name()+":"+this.regionName, true);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
index 85a55a7..1c8586f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CacheServerMBeanShiroJUnitTest.java
@@ -83,7 +83,7 @@ public class CacheServerMBeanShiroJUnitTest {
 
   @Test
   @JMXConnectionConfiguration(user = "dataReader", password = "12345")
-  public void testDataRead() throws Exception{
+  public void ztestDataRead() throws Exception{
     assertThatThrownBy(() -> bean.removeIndex("foo")).hasMessageContaining(TestCommand.dataManage.toString());
     assertThatThrownBy(() -> bean.fetchLoadProbe()).hasMessageContaining(TestCommand.clusterRead.toString());
     assertThatThrownBy(() -> bean.getActiveCQCount()).hasMessageContaining(TestCommand.clusterRead.toString());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
index 97260d8..9c9b4fc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/DataCommandsSecurityTest.java
@@ -74,10 +74,10 @@ public class DataCommandsSecurityTest {
     assertThatThrownBy(() -> bean.processCommand("import data --region=region2 --file=foo.txt --member=value")).isInstanceOf(GemFireSecurityException.class);
 
     assertThatThrownBy(() -> bean.processCommand("put --key=key1 --value=value1 --region=region2")).isInstanceOf(GemFireSecurityException.class)
-        .hasMessageContaining("[data]:[write]:[region2]");
+        .hasMessageContaining("DATA:WRITE:region2");
 
     assertThatThrownBy(() -> bean.processCommand("get --key=key1 --region=region2")).isInstanceOf(GemFireSecurityException.class)
-        .hasMessageContaining("[data]:[read]:[region2]");
+        .hasMessageContaining("DATA:READ:region2");
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
index 0bf3cab..52f37e6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilCustomRealmJUnitTest.java
@@ -17,16 +17,10 @@
 
 package com.gemstone.gemfire.management.internal.security;
 
-import java.util.Properties;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.security.shiro.CustomAuthRealm;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.mgt.DefaultSecurityManager;
-import org.apache.shiro.mgt.SecurityManager;
-import org.apache.shiro.realm.Realm;
 import org.junit.BeforeClass;
 import org.junit.experimental.categories.Category;
 
@@ -39,14 +33,10 @@ import org.junit.experimental.categories.Category;
 public class GeodeSecurityUtilCustomRealmJUnitTest extends GeodeSecurityUtilWithIniFileJUnitTest {
   @BeforeClass
   public static void beforeClass() throws Exception{
-    Properties properties = new Properties();
-    properties.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, JSONAuthorization.class.getName() + ".create");
-    properties.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
+    props.put(DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME, JSONAuthorization.class.getName() + ".create");
+    props.put(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME, JSONAuthorization.class.getName() + ".create");
     JSONAuthorization.setUpWithJsonFile("shiro-ini.json");
-
-    Realm realm = new CustomAuthRealm(properties);
-    SecurityManager securityManager = new DefaultSecurityManager(realm);
-    SecurityUtils.setSecurityManager(securityManager);
+    GeodeSecurityUtil.initSecurity(props);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
index fe80180..63bf447 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GeodeSecurityUtilWithIniFileJUnitTest.java
@@ -19,14 +19,14 @@ package com.gemstone.gemfire.management.internal.security;
 
 import static org.assertj.core.api.Assertions.*;
 
+import java.util.Properties;
+
 import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
+import com.gemstone.gemfire.security.GemFireSecurityException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.config.IniSecurityManagerFactory;
-import org.apache.shiro.mgt.SecurityManager;
 import org.apache.shiro.util.ThreadContext;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -39,12 +39,11 @@ import org.junit.experimental.categories.Category;
  */
 @Category(UnitTest.class)
 public class GeodeSecurityUtilWithIniFileJUnitTest {
+  protected static Properties props = new Properties();
   @BeforeClass
   public static void beforeClass() throws Exception{
-    ThreadContext.remove();
-    IniSecurityManagerFactory factory = new IniSecurityManagerFactory("classpath:shiro.ini");
-    SecurityManager securityManager = factory.getInstance();
-    SecurityUtils.setSecurityManager(securityManager);
+    props.setProperty(DistributionConfig.SECURITY_SHIRO_INIT_NAME, "shiro.ini");
+    GeodeSecurityUtil.initSecurity(props);
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index 8eaaf6a..377ab77 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -112,7 +112,7 @@ public class GfshCommandsSecurityTest {
 
 
   private void runCommandsWithAndWithout(String permission) throws Exception{
-    List<TestCommand> permitted = TestCommand.getPermittedCommands(new WildcardPermission(permission));
+    List<TestCommand> permitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
     for(TestCommand clusterRead:permitted) {
       LogService.getLogger().info("Processing authorized command: "+clusterRead.getCommand());gfsh.executeCommand(clusterRead.getCommand());
       CommandResult result = (CommandResult) gfsh.getResult();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
index 9e2e41a..ec89aaa 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ResourceOperationContextJUnitTest.java
@@ -61,6 +61,11 @@ public class ResourceOperationContextJUnitTest {
     context = new ResourceOperationContext("DATA", null, null);
     assertEquals(Resource.DATA, context.getResource());
     assertEquals(OperationCode.NULL, context.getOperationCode());
+    assertEquals("NULL", context.getRegionName());
+
+    context = new ResourceOperationContext("CLUSTER", null, null);
+    assertEquals(Resource.CLUSTER, context.getResource());
+    assertEquals(OperationCode.NULL, context.getOperationCode());
     assertEquals(null, context.getRegionName());
 
     context = new ResourceOperationContext(null, "MANAGE", "REGIONA");
@@ -77,12 +82,12 @@ public class ResourceOperationContextJUnitTest {
   @Test
   public void testToString(){
     context = new ResourceOperationContext();
-    assertEquals("[null]:[null]:[null]", context.toString());
+    assertEquals("NULL:NULL", context.toString());
 
     context = new ResourceOperationContext("DATA", "MANAGE");
-    assertEquals("[data]:[manage]:[null]", context.toString());
+    assertEquals("DATA:MANAGE:NULL", context.toString());
 
     context = new ResourceOperationContext("DATA", "MANAGE", "REGIONA");
-    assertEquals("[data]:[manage]:[regiona]", context.toString());
+    assertEquals("DATA:MANAGE:REGIONA", context.toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
index 7d683f3..f4c2e06 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/ShiroCacheStartRule.java
@@ -43,7 +43,7 @@ public class ShiroCacheStartRule extends ExternalResource {
     properties.put(DistributionConfig.JMX_MANAGER_START_NAME, "true");
     properties.put(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxManagerPort));
     properties.put(DistributionConfig.HTTP_SERVICE_PORT_NAME, "0");
-    properties.put(DistributionConfig.SHIRO_INIT_NAME, shiroFile);
+    properties.put(DistributionConfig.SECURITY_SHIRO_INIT_NAME, shiroFile);
 
     cache = new CacheFactory(properties).create();
     cache.addCacheServer().start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/84a9eb1b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
index 56eeeec..667330c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
@@ -96,14 +96,14 @@ public class TestCommand {
     createTestCommand("destroy region --name=value", dataManage);
 
     //Data Commands
-    createTestCommand("rebalance --include-region=regionA", dataManage);
-    createTestCommand("export data --region=regionA --file=export.txt --member=exportMember", regionARead);
-    createTestCommand("import data --region=regionA --file=import.txt --member=importMember", regionAWrite);
-    createTestCommand("put --key=key1 --value=value1 --region=regionA", regionAWrite);
-    createTestCommand("get --key=key1 --region=regionA", regionARead);
-    createTestCommand("remove --region=regionA", dataManage);
-    createTestCommand("query --query='SELECT * FROM /region1'", dataRead);
-    createTestCommand("locate entry --key=k1 --region=regionA", regionARead);
+    createTestCommand("rebalance --include-region=RegionA", dataManage);
+    createTestCommand("export data --region=RegionA --file=export.txt --member=exportMember", regionARead);
+    createTestCommand("import data --region=RegionA --file=import.txt --member=importMember", regionAWrite);
+    createTestCommand("put --key=key1 --value=value1 --region=RegionA", regionAWrite);
+    createTestCommand("get --key=key1 --region=RegionA", regionARead);
+    createTestCommand("remove --region=RegionA", dataManage);
+    createTestCommand("query --query='SELECT * FROM /RegionA'", dataRead);
+    createTestCommand("locate entry --key=k1 --region=RegionA", regionARead);
 
     // Deploy commands
     //createTestCommand("deploy --jar=group1_functions.jar --group=Group1", dataManage); // TODO: this command will fail in GfshCommandsSecurityTest at interceptor for jar file checking


[15/50] [abbrv] incubator-geode git commit: GEODE-1222: remove InsufficientDiskSpaceException

Posted by kl...@apache.org.
GEODE-1222: remove InsufficientDiskSpaceException


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

Branch: refs/heads/feature/GEODE-835
Commit: e1cbc9b25da6353b62e7d59c67babc470e81eb1f
Parents: b6de5a3
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed May 11 16:05:48 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon May 16 11:29:06 2016 -0700

----------------------------------------------------------------------
 .../InsufficientDiskSpaceException.java         | 53 --------------------
 .../gemstone/gemfire/internal/cache/Oplog.java  |  7 ++-
 .../codeAnalysis/sanctionedSerializables.txt    |  1 -
 3 files changed, 3 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e1cbc9b2/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java
deleted file mode 100644
index be87c8b..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/InsufficientDiskSpaceException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import com.gemstone.gemfire.cache.DiskAccessException;
-import com.gemstone.gemfire.cache.DiskStore;
-import com.gemstone.gemfire.internal.Sendable;
-
-/**
- * @since gfxd 1.0.1
- */
-public class InsufficientDiskSpaceException extends DiskAccessException implements Sendable {
-  private static final long serialVersionUID = -6167707908956900841L;
-
-  public InsufficientDiskSpaceException(String msg, Throwable cause, DiskStore ds) {
-    super(msg, cause, ds);
-  }
-
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public void sendTo(DataOutput out) throws IOException {
-    // send base DiskAccessException to older versions
-    Version peerVersion = InternalDataSerializer.getVersionForDataStream(out);
-    if (Version.GFE_80.compareTo(peerVersion) > 0) {
-      DiskAccessException dae = new DiskAccessException(getMessage(),
-          getCause());
-      InternalDataSerializer.writeSerializableObject(dae, out);
-    }
-    else {
-      InternalDataSerializer.writeSerializableObject(this, out);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e1cbc9b2/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
index 4728594..96f616b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
@@ -77,7 +77,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.FileUtil;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.InsufficientDiskSpaceException;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalStatisticsDisabledException;
 import com.gemstone.gemfire.internal.Sendable;
@@ -603,7 +602,7 @@ public final class Oplog implements CompactableOplog, Flushable {
     long availableSpace = this.dirHolder.getAvailableSpace();
     if (availableSpace < maxOplogSizeParam) {
       if (DiskStoreImpl.PREALLOCATE_OPLOGS && !DiskStoreImpl.SET_IGNORE_PREALLOCATE) {
-        throw new InsufficientDiskSpaceException(LocalizedStrings.Oplog_PreAllocate_Failure_Init.toLocalizedString(this.dirHolder,
+        throw new DiskAccessException(LocalizedStrings.Oplog_PreAllocate_Failure_Init.toLocalizedString(this.dirHolder,
             maxOplogSizeParam), new IOException("not enough space left to create and pre grow oplog files, available="
             + availableSpace + ", required=" + maxOplogSizeParam), getParent());
       }
@@ -1070,13 +1069,13 @@ public final class Oplog implements CompactableOplog, Flushable {
          * false; } } }
          */
         closeAndDeleteAfterEx(ioe, olf);
-        throw new InsufficientDiskSpaceException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(olf.f
+        throw new DiskAccessException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(olf.f
             .getAbsolutePath(), maxSize), ioe, getParent());
       }
     }
     // TODO: Perhaps the test flag is not requierd here. Will re-visit.
     else if (DiskStoreImpl.PREALLOCATE_OPLOGS && !DiskStoreImpl.SET_IGNORE_PREALLOCATE) {
-      throw new InsufficientDiskSpaceException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(
+      throw new DiskAccessException(LocalizedStrings.Oplog_PreAllocate_Failure.toLocalizedString(
           olf.f.getAbsolutePath(), maxSize), new IOException("not enough space left to pre-blow, available=" + availableSpace
           + ", required=" + maxSize), getParent());
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e1cbc9b2/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 fa02ca3..1ee16d5 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
@@ -246,7 +246,6 @@ com/gemstone/gemfire/internal/ConfigSource$Type,false
 com/gemstone/gemfire/internal/CopyOnWriteHashSet,true,8591978652141659932
 com/gemstone/gemfire/internal/DSFIDFactory$SqlfSerializationException,true,5076687296705595933
 com/gemstone/gemfire/internal/DSFIDNotFoundException,true,130596009484324655,dsfid:int,versionOrdinal:short
-com/gemstone/gemfire/internal/InsufficientDiskSpaceException,true,-6167707908956900841
 com/gemstone/gemfire/internal/InternalDataSerializer$SERIALIZATION_VERSION,false
 com/gemstone/gemfire/internal/InternalStatisticsDisabledException,true,4146181546364258311
 com/gemstone/gemfire/internal/LinuxProcFsStatistics$CPU,false


[28/50] [abbrv] incubator-geode git commit: GEODE-1293: Revert some changes which broke publishing

Posted by kl...@apache.org.
GEODE-1293: Revert some changes which broke publishing

- Reinstate 'classifier' for the internal modules jar. This is still
  broken as the published jars are not correctly named. Will need to fix
  this in the near future.


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

Branch: refs/heads/feature/GEODE-835
Commit: 00c794b4cbc186da419cce28c1bf62a977bf2829
Parents: fcd5491
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 18 08:40:52 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed May 18 08:40:52 2016 -0700

----------------------------------------------------------------------
 extensions/geode-modules-session/build.gradle | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00c794b4/extensions/geode-modules-session/build.gradle
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/build.gradle b/extensions/geode-modules-session/build.gradle
index 765bdd9..5f25d97 100644
--- a/extensions/geode-modules-session/build.gradle
+++ b/extensions/geode-modules-session/build.gradle
@@ -45,6 +45,7 @@ task internalJar(type: Jar) {
   from sourceSets.main.output
   include '**/internal/**/*'
   includeEmptyDirs = false
+  classifier = 'internal'
   baseName = 'geode-modules-session-internal'
 }
 


[17/50] [abbrv] incubator-geode git commit: GEODE-1351: let the waitForFlush() to return false if timeout

Posted by kl...@apache.org.
GEODE-1351: let the waitForFlush() to return false if timeout

also apply it on some unit tests.


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

Branch: refs/heads/feature/GEODE-835
Commit: 3e8a610e99bc005bcc56c378e3d1e2274a3d468c
Parents: 34d3791
Author: zhouxh <gz...@pivotal.io>
Authored: Mon May 16 17:12:15 2016 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Mon May 16 17:12:15 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneIndex.java       |  4 +++-
 .../cache/lucene/internal/LuceneIndexImpl.java  | 11 ++++-----
 .../internal/xml/LuceneIndexCreation.java       |  3 ++-
 .../LuceneIndexCreationIntegrationTest.java     | 25 ++++++++++----------
 .../LuceneIndexRecoveryHAIntegrationTest.java   | 14 +++++------
 5 files changed, 29 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8a610e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
index be329f7..6b1a4b4 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneIndex.java
@@ -58,7 +58,9 @@ public interface LuceneIndex {
   
   /* 
    * wait until the current entries in cache are indexed
+   * @param maxWaitInMilliseconds max wait time in millisecond
+   * @return if entries are flushed within maxWait
    */
-  public void waitUntilFlushed(int maxWait);
+  public boolean waitUntilFlushed(int maxWaitInMillisecond);
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8a610e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
index 981d9e4..c165085 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexImpl.java
@@ -81,19 +81,17 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     searchableFieldNames = fields;
   }
 
-  /*
-   *  For test and demo purpose. To use it, the data region should stop feeding
-   *  A more advanced version is under-development
-   */
   @Override
-  public void waitUntilFlushed(int maxWait) {
+  public boolean waitUntilFlushed(int maxWaitInMillisecond) {
     String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, regionPath);
     AsyncEventQueue queue = (AsyncEventQueue)cache.getAsyncEventQueue(aeqId);
+    boolean flushed = false;
     if (queue != null) {
       long start = System.nanoTime();
-      while (System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(maxWait)) {
+      while (System.nanoTime() - start < TimeUnit.MILLISECONDS.toNanos(maxWaitInMillisecond)) {
         if (0 == queue.size()) {
           logger.debug("waitUntilFlushed: Queue size is 0");
+          flushed = true;
           break;
         } else {
           try {
@@ -105,6 +103,7 @@ public abstract class LuceneIndexImpl implements InternalLuceneIndex {
     } else { 
       throw new IllegalArgumentException("The AEQ does not exist for the index "+indexName+" region "+regionPath);
     }
+    return flushed;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8a610e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
index b54f51b..a3bdd24 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/xml/LuceneIndexCreation.java
@@ -116,6 +116,7 @@ public class LuceneIndexCreation implements LuceneIndex, Extension<Region<?, ?>>
   }
 
   @Override
-  public void waitUntilFlushed(int maxWait) {
+  public boolean waitUntilFlushed(int maxWaitInMillisecond) {
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8a610e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 6429143..fe754a4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -44,7 +44,6 @@ import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
@@ -71,12 +70,10 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     Region region = createRegion();
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     region.put("key1", new TestObject());
-
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     assertEquals(analyzers, index.getFieldAnalyzers());
-    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
-      assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
-      assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
-    });
+    assertEquals(Arrays.asList("field1"), field1Analyzer.analyzedfields);
+    assertEquals(Arrays.asList("field2"), field2Analyzer.analyzedfields);
   }
 
   @Test
@@ -188,26 +185,28 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     cache.close();
     createCache();
     createIndex("field1", "field2");
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
     LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
       .create(INDEX_NAME, REGION_NAME,
         "field1:world");
-    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
-      assertEquals(1, query.search().size());
-    });
+    assertEquals(1, query.search().size());
   }
 
+  private void verifyIndexFinishFlushing(String indexName, String regionName) {
+    LuceneIndex index = luceneService.getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
+  }
+  
   @Test
   public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
     createIndex("field1", "field2");
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
     dataRegion.put("A", new TestObject());
-    final AsyncEventQueueImpl queue = (AsyncEventQueueImpl) getIndexQueue();
-
-    //Wait until the queue has drained
-    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, queue.size()));
+    verifyIndexFinishFlushing(INDEX_NAME, REGION_NAME);
     cache.close();
     createCache();
     createIndex("text");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3e8a610e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
index 77d2a5c..d32e6d8 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
@@ -41,6 +41,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
@@ -55,7 +56,6 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import com.jayway.awaitility.Awaitility;
 
 @Category(IntegrationTest.class)
 public class LuceneIndexRecoveryHAIntegrationTest {
@@ -139,7 +139,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     value = new Type1("lucene world", 1, 2L, 3.0, 4.0f);
     userRegion.put("value3", value);
 
-    waitUntilQueueEmpty(aeqId);
+    verifyIndexFinishFlushing(INDEX, REGION);
 
     LuceneQuery<Integer, Type1> query = service.createLuceneQueryFactory().create(INDEX, REGION, "s:world");
     LuceneQueryResults<Integer, Type1> results = query.search();
@@ -190,7 +190,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     value = new Type1("lucene world", 1, 2L, 3.0, 4.0f);
     userRegion.put("value3", value);
 
-    waitUntilQueueEmpty(aeqId);
+    verifyIndexFinishFlushing(INDEX, REGION);
 
     PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
     assertNotNull(fileRegion);
@@ -203,9 +203,9 @@ public class LuceneIndexRecoveryHAIntegrationTest {
     Assert.assertEquals(3, results.size());
   }
 
-  private void waitUntilQueueEmpty(final String aeqId) {
-    // TODO flush queue
-    AsyncEventQueue queue = cache.getAsyncEventQueue(aeqId);
-    Awaitility.waitAtMost(1000, TimeUnit.MILLISECONDS).until(() -> assertEquals(0, queue.size()));
+  private void verifyIndexFinishFlushing(String indexName, String regionName) {
+    LuceneIndex index = LuceneServiceProvider.get(cache).getIndex(indexName, regionName);
+    boolean flushed = index.waitUntilFlushed(60000);
+    assertTrue(flushed);
   }
 }


[29/50] [abbrv] incubator-geode git commit: GEODE-1133 SeparateClassloaderTestRunner has to be re-implemented

Posted by kl...@apache.org.
GEODE-1133 SeparateClassloaderTestRunner has to be re-implemented

Remove SeparateClassloaderTestRunner since it is not used


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

Branch: refs/heads/feature/GEODE-835
Commit: b1ff33dd6e5a095bf3b0ca4f3791f2e651676db8
Parents: 00c794b
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Wed May 18 09:02:26 2016 -0700
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Wed May 18 09:02:26 2016 -0700

----------------------------------------------------------------------
 .../junit/SeparateClassloaderTestRunner.java    | 56 --------------------
 1 file changed, 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b1ff33dd/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/junit/SeparateClassloaderTestRunner.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/junit/SeparateClassloaderTestRunner.java b/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/junit/SeparateClassloaderTestRunner.java
deleted file mode 100644
index 4337f5a..0000000
--- a/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/junit/SeparateClassloaderTestRunner.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
-* 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.modules.session.junit;
-
-import org.junit.runners.BlockJUnit4ClassRunner;
-import org.junit.runners.model.InitializationError;
-
-import java.net.URLClassLoader;
-
-/**
- * @author StackOverflow
- */
-public class SeparateClassloaderTestRunner extends BlockJUnit4ClassRunner {
-
-  public SeparateClassloaderTestRunner(Class<?> clazz) throws InitializationError {
-    super(getFromTestClassloader(clazz));
-  }
-
-  private static Class<?> getFromTestClassloader(Class<?> clazz) throws InitializationError {
-    try {
-      ClassLoader testClassLoader = new TestClassLoader();
-      return Class.forName(clazz.getName(), true, testClassLoader);
-    } catch (ClassNotFoundException e) {
-      throw new InitializationError(e);
-    }
-  }
-
-  public static class TestClassLoader extends URLClassLoader {
-    public TestClassLoader() {
-      super(((URLClassLoader)getSystemClassLoader()).getURLs());
-    }
-
-    @Override
-    public Class<?> loadClass(String name) throws ClassNotFoundException {
-      if (name.startsWith("com.gemstone.gemfire.modules.session.")) {
-        return super.findClass(name);
-      }
-      return super.loadClass(name);
-    }
-  }
-}


[49/50] [abbrv] incubator-geode git commit: GEODE-17: add regionName in some of the commands authorization

Posted by kl...@apache.org.
GEODE-17: add regionName in some of the commands authorization


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

Branch: refs/heads/feature/GEODE-835
Commit: 8a3c351abd971ba740e77364a431aee7d5cc5b10
Parents: f05e87d
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu May 19 10:39:16 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri May 20 09:36:10 2016 -0700

----------------------------------------------------------------------
 .../internal/security/GeodeSecurityUtil.java    |   4 +
 .../CreateAlterDestroyRegionCommands.java       |   5 +-
 .../internal/cli/commands/DataCommands.java     |   1 -
 .../internal/cli/commands/IndexCommands.java    |  32 +-
 .../cli/functions/DataCommandFunction.java      |  11 +-
 .../support/MemberMXBeanAdapter.java            | 654 -------------------
 .../security/CliCommandsSecurityTest.java       |  24 +-
 .../security/GfshCommandsSecurityTest.java      |  22 +-
 .../internal/security/TestCommand.java          |   9 +-
 .../internal/security/cacheServer.json          |  42 ++
 10 files changed, 118 insertions(+), 686 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 322c59e..4eafada 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -173,6 +173,10 @@ public class GeodeSecurityUtil {
     authorize("DATA", "READ");
   }
 
+  public static void authorizeRegionManage(String regionName){
+    authorize("DATA", "MANAGE", regionName);
+  }
+
   public static void authorizeRegionWrite(String regionName){
     authorize("DATA", "WRITE", regionName);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index 06c096f..1e19861 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -52,6 +52,7 @@ import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.DistributedRegionMXBean;
 import com.gemstone.gemfire.management.DistributedSystemMXBean;
 import com.gemstone.gemfire.management.ManagementService;
@@ -437,7 +438,6 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
   
   @CliCommand (value = CliStrings.ALTER_REGION, help = CliStrings.ALTER_REGION__HELP)
   @CliMetaData (relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION, writesToSharedConfiguration = true)
-  @ResourceOperation(resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result alterRegion(
       @CliOption (key = CliStrings.ALTER_REGION__REGION,
                   mandatory = true,
@@ -531,6 +531,8 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
     Result result = null;
     XmlEntity xmlEntity = null;
 
+    GeodeSecurityUtil.authorizeRegionManage(regionPath);
+
     try {
       Cache cache = CacheFactory.getAnyInstance();
 
@@ -1003,6 +1005,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
           mandatory = true,
           help = CliStrings.DESTROY_REGION__REGION__HELP)
       String regionPath) {
+
     if (regionPath == null) {
       return ResultBuilder.createInfoResult(CliStrings.DESTROY_REGION__MSG__SPECIFY_REGIONPATH_TO_DESTROY);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
index fafea9a..778089e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
@@ -1173,7 +1173,6 @@ public class DataCommands implements CommandMarker {
       CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
   @MultiStepCommand
   @CliCommand(value = { CliStrings.QUERY }, help = CliStrings.QUERY__HELP)
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.READ)
   public Object query(
       @CliOption(key = CliStrings.QUERY__QUERY, help = CliStrings.QUERY__QUERY__HELP, mandatory = true) final String query,
       @CliOption(key = CliStrings.QUERY__STEPNAME, mandatory = false, help = "Step name", unspecifiedDefaultValue = CliStrings.QUERY__STEPNAME__DEFAULTVALUE) String stepName,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
index 24f9f71..b863737 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.cache.operations.OperationContext.*;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -26,6 +37,7 @@ import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.execute.AbstractExecution;
 import com.gemstone.gemfire.internal.lang.StringUtils;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.cli.CliMetaData;
 import com.gemstone.gemfire.management.cli.ConverterHint;
 import com.gemstone.gemfire.management.cli.Result;
@@ -47,22 +59,11 @@ import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationWriter;
 import com.gemstone.gemfire.management.internal.configuration.domain.XmlEntity;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
+
 import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-
-import static com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import static com.gemstone.gemfire.cache.operations.OperationContext.Resource;
-
 /**
  * The IndexCommands class encapsulates all GemFire shell (Gfsh) commands related to indexes defined in GemFire.
  * </p>
@@ -171,7 +172,6 @@ public class IndexCommands extends AbstractCommandsSupport {
 
   @CliCommand(value = CliStrings.CREATE_INDEX, help = CliStrings.CREATE_INDEX__HELP)
   @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO : Add optionContext for indexName
   public Result createIndex(
       @CliOption (key = CliStrings.CREATE_INDEX__NAME,
@@ -205,7 +205,8 @@ public class IndexCommands extends AbstractCommandsSupport {
 
     Result result = null;
     XmlEntity xmlEntity = null;
-    
+
+    GeodeSecurityUtil.authorizeRegionManage(regionPath);
     try {
       final Cache cache = CacheFactory.getAnyInstance();
 
@@ -458,7 +459,6 @@ public class IndexCommands extends AbstractCommandsSupport {
 
   @CliCommand(value = CliStrings.DEFINE_INDEX, help = CliStrings.DEFINE_INDEX__HELP)
   @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
-  @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO : Add optionContext for indexName
   public Result defineIndex(
       @CliOption (key = CliStrings.DEFINE_INDEX_NAME,
@@ -482,6 +482,8 @@ public class IndexCommands extends AbstractCommandsSupport {
 
     Result result = null;
     XmlEntity xmlEntity = null;
+
+    GeodeSecurityUtil.authorizeRegionManage(regionPath);
     
     int idxType = IndexInfo.RANGE_INDEX;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
index b4e7340..ace24f2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
@@ -26,8 +26,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -59,6 +57,7 @@ import com.gemstone.gemfire.internal.InternalEntity;
 import com.gemstone.gemfire.internal.NanoTimer;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.commands.DataCommands;
@@ -77,6 +76,8 @@ import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.JsonUtil;
 import com.gemstone.gemfire.pdx.PdxInstance;
+
+import org.apache.logging.log4j.Logger;
 import org.json.JSONArray;
 
 /***
@@ -912,6 +913,12 @@ public class DataCommandFunction extends FunctionAdapter implements  InternalEnt
         CompiledValue compiledQuery = compiler.compileQuery(query);
         Set<String> regions = new HashSet<String>();
         compiledQuery.getRegionsInQuery(regions, null);
+
+        // authorize data read on these regions
+        for(String region:regions){
+          GeodeSecurityUtil.authorizeRegionRead(region);
+        }
+
         regionsInQuery = Collections.unmodifiableSet(regions);
         if (regionsInQuery.size() > 0) {
           Set<DistributedMember> members = DataCommands.getQueryRegionsAssociatedMembers(regionsInQuery, cache, false);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
deleted file mode 100644
index a3d4cd0..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
+++ /dev/null
@@ -1,654 +0,0 @@
-/*
- * 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.management.internal.web.controllers.support;
-
-import java.util.Map;
-
-import com.gemstone.gemfire.management.GemFireProperties;
-import com.gemstone.gemfire.management.JVMMetrics;
-import com.gemstone.gemfire.management.MemberMXBean;
-import com.gemstone.gemfire.management.OSMetrics;
-
-/**
- * The MemberMXBeanAdapter class is an abstract adapter class to the MemberMXBean interface.
- * <p/>
- * @see com.gemstone.gemfire.management.MemberMXBean
- * @since 8.0
- */
-public class MemberMXBeanAdapter implements MemberMXBean {
-
-  @Override
-  public String showLog(final int numberOfLines) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String viewLicense() { throw new UnsupportedOperationException("Not Implemented!"); }
-
-  @Override
-  public String[] compactAllDiskStores() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean createManager() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public void shutDownMember() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public JVMMetrics showJVMMetrics() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public OSMetrics showOSMetrics() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String processCommand(final String commandString) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String processCommand(final String commandString, final Map<String, String> env) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String processCommand(final String commandString, final Map<String, String> env, final Byte[][] binaryData) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listDiskStores(final boolean includeRegionOwned) {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-  
-  @Override
-  public GemFireProperties listGemFireProperties() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getHost() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getName() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getId() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getMember() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] getGroups() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getProcessId() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String status() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getVersion() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isLocator() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getLockTimeout() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getLockLease() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isServer() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean hasGatewaySender() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isManager() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean isManagerCreated() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public boolean hasGatewayReceiver() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String getClassPath() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getCurrentTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getMemberUpTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getCpuUsage() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  @Deprecated
-  public long getCurrentHeapSize() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  @Deprecated
-  public long getMaximumHeapSize() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  @Deprecated
-  public long getFreeHeapSize() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] fetchJvmThreads() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getFileDescriptorLimit() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTotalFileDescriptorOpen() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalRegionCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getPartitionRegionCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listRegions() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] getDiskStores() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] getRootRegionNames() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalRegionEntryCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalBucketCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalPrimaryBucketCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getGetsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getPutsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getPutAllAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalMissCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalHitCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getGetsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getPutsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getPutAllRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getCreatesRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDestroysRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getCacheWriterCallsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getCacheListenerCallsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalLoadsCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getLoadsAverageLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNetLoadsCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getNetLoadsAverageLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNetSearchCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getNetSearchAverageLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalDiskTasksWaiting() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getBytesSentRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getBytesReceivedRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listConnectedGatewayReceivers() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public String[] listConnectedGatewaySenders() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getNumRunningFunctions() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getFunctionExecutionRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getNumRunningFunctionsHavingResults() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalTransactionsCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTransactionCommitsAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTransactionCommittedTotalCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTransactionRolledBackTotalCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getTransactionCommitsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDiskReadsRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDiskWritesRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getDiskFlushAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalBackupInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalBackupCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getLockWaitsInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTotalLockWaitTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNumberOfLockService() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getTotalNumberOfGrantors() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getLockRequestQueues() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getLruEvictionRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getLruDestroyRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getInitialImagesInProgres() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getInitialImageTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getInitialImageKeysReceived() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getDeserializationAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getDeserializationLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getDeserializationRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getSerializationAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getSerializationLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getSerializationRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getPDXDeserializationRate() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getPDXDeserializationAvgLatency() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getTotalDiskUsage() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getNumThreads() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public double getLoadAverage() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getGarbageCollectionCount() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getGarbageCollectionTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getAverageReads() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public float getAverageWrites() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getJVMPauses() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override 
-    public int getHostCpuUsage() { 
-       throw new UnsupportedOperationException("Not Implemented!"); 
-   }
-
-  @Override
-  public boolean isCacheServer() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public String getRedundancyZone() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getRebalancesInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getReplyWaitsInProgress() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getReplyWaitsCompleted() {
-    throw new UnsupportedOperationException("Not Implemented!"); 
-  }
-
-  @Override
-  public int getVisibleNodes() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getOffHeapObjects() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapMaxMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapFreeMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapUsedMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public int getOffHeapFragmentation() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-
-  @Override
-  public long getOffHeapCompactionTime() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }    
-
-  @Override
-  public long getMaxMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-  
-  @Override
-  public long getFreeMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-  
-  @Override
-  public long getUsedMemory() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
index 0864e52..3ccd71c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/CliCommandsSecurityTest.java
@@ -16,13 +16,15 @@
  */
 package com.gemstone.gemfire.management.internal.security;
 
-import static org.assertj.core.api.AssertionsForClassTypes.*;
+import static org.assertj.core.api.AssertionsForClassTypes.fail;
+import static org.junit.Assert.*;
 
 import java.util.List;
 
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.MemberMXBean;
+import com.gemstone.gemfire.security.NotAuthorizedException;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 import org.junit.Before;
@@ -58,14 +60,20 @@ public class CliCommandsSecurityTest {
   @JMXConnectionConfiguration(user = "stranger", password = "1234567")
   public void testNoAccess(){
    for (TestCommand command:commands) {
+     // skip query commands since query commands are only available in client shell
+     if(command.getCommand().startsWith("query"))
+       continue;
       LogService.getLogger().info("processing: "+command.getCommand());
-      // for those commands that don't require any permission, any user can execute them
-      if(command.getPermission()==null){
-        bean.processCommand(command.getCommand());
-      }
-      else {
-        assertThatThrownBy(() -> bean.processCommand(command.getCommand()))
-            .hasMessageContaining(command.getPermission().toString());
+      // for those commands that requires a permission, we expect an exception to be thrown
+      if(command.getPermission()!=null){
+        try{
+          String result = bean.processCommand(command.getCommand());
+          fail(command.getCommand() + " has result: "+ result);
+        }
+        catch(NotAuthorizedException e){
+          assertTrue(e.getMessage()+" should contain "+command.getPermission(),
+            e.getMessage().contains(command.getPermission().toString()));
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
index b21302e..1a15367 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/GfshCommandsSecurityTest.java
@@ -110,6 +110,24 @@ public class GfshCommandsSecurityTest {
     runCommandsWithAndWithout("DATA:MANAGE");
   }
 
+  @Test
+  @JMXConnectionConfiguration(user = "regionA-reader", password = "1234567")
+  public void testregionAReader() throws Exception{
+    runCommandsWithAndWithout("DATA:READ:RegionA");
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "regionA-writer", password = "1234567")
+  public void testregionAWriter() throws Exception{
+    runCommandsWithAndWithout("DATA:WRITE:RegionA");
+  }
+
+  @Test
+  @JMXConnectionConfiguration(user = "regionA-manager", password = "1234567")
+  public void testregionAManager() throws Exception{
+    runCommandsWithAndWithout("DATA:MANAGE:RegionA");
+  }
+
 
   private void runCommandsWithAndWithout(String permission) throws Exception{
     List<TestCommand> allPermitted = TestCommand.getPermittedCommands(new WildcardPermission(permission, true));
@@ -148,7 +166,9 @@ public class GfshCommandsSecurityTest {
       }
 
       assertEquals(ResultBuilder.ERRORCODE_UNAUTHORIZED, ((ErrorResultData) result.getResultData()).getErrorCode());
-      assertTrue(result.getContent().toString().contains(other.getPermission().toString()));
+      String resultMessage = result.getContent().toString();
+      String permString = other.getPermission().toString();
+      assertTrue(resultMessage+" does not contain "+permString,resultMessage.contains(permString));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
index 667330c..4b482a9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/security/TestCommand.java
@@ -33,6 +33,7 @@ public class TestCommand {
 
   public static OperationContext regionARead = new ResourceOperationContext("DATA", "READ", "RegionA");
   public static OperationContext regionAWrite = new ResourceOperationContext("DATA", "WRITE", "RegionA");
+  public static OperationContext regionAManage = new ResourceOperationContext("DATA", "MANAGE", "RegionA");
 
   public static OperationContext clusterRead = new ResourceOperationContext("CLUSTER", "READ");
   public static OperationContext clusterWrite = new ResourceOperationContext("CLUSTER", "WRITE");
@@ -91,7 +92,7 @@ public class TestCommand {
     createTestCommand("export config --member=member1", clusterRead);
 
     //CreateAlterDestroyRegionCommands
-    createTestCommand("alter region --name=region1 --eviction-max=5000", dataManage);
+    createTestCommand("alter region --name=RegionA --eviction-max=5000", regionAManage);
     createTestCommand("create region --name=region12 --type=REPLICATE", dataManage);
     createTestCommand("destroy region --name=value", dataManage);
 
@@ -102,7 +103,7 @@ public class TestCommand {
     createTestCommand("put --key=key1 --value=value1 --region=RegionA", regionAWrite);
     createTestCommand("get --key=key1 --region=RegionA", regionARead);
     createTestCommand("remove --region=RegionA", dataManage);
-    createTestCommand("query --query='SELECT * FROM /RegionA'", dataRead);
+    createTestCommand("query --query='SELECT * FROM /RegionA'", regionARead);
     createTestCommand("locate entry --key=k1 --region=RegionA", regionARead);
 
     // Deploy commands
@@ -147,8 +148,8 @@ public class TestCommand {
     //IndexCommands
     createTestCommand("clear defined indexes", dataManage);
     createTestCommand("create defined indexes", dataManage);
-    createTestCommand("create index --name=myKeyIndex --expression=region1.Id --region=region1 --type=key", dataManage);
-    createTestCommand("define index --name=myIndex1 --expression=exp1 --region=/exampleRegion", dataManage);
+    createTestCommand("create index --name=myKeyIndex --expression=region1.Id --region=RegionA --type=key", regionAManage);
+    createTestCommand("define index --name=myIndex1 --expression=exp1 --region=/RegionA", regionAManage);
     createTestCommand("destroy index --member=server2", dataManage);
     createTestCommand("list indexes", clusterRead);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a3c351a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
index 638ae07..fbbda8d 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/cacheServer.json
@@ -82,6 +82,27 @@
       "regions": "null,region1"
     },
     {
+      "name": "regionA-manage",
+      "operationsAllowed": [
+        "DATA:MANAGE"
+      ],
+      "regions": "RegionA"
+    },
+    {
+      "name": "regionA-write",
+      "operationsAllowed": [
+        "DATA:WRITE"
+      ],
+      "regions": "RegionA"
+    },
+    {
+      "name": "regionA-read",
+      "operationsAllowed": [
+        "DATA:READ"
+      ],
+      "regions": "RegionA"
+    },
+    {
       "name": "secure-use",
       "operationsAllowed": [
         "DATA:READ",
@@ -183,6 +204,27 @@
       "roles": [
         "cluster-manage"
       ]
+    },
+    {
+      "name": "regionA-manager",
+      "password": "1234567",
+      "roles": [
+        "regionA-manage"
+      ]
+    },
+    {
+      "name": "regionA-writer",
+      "password": "1234567",
+      "roles": [
+        "regionA-write"
+      ]
+    },
+    {
+      "name": "regionA-reader",
+      "password": "1234567",
+      "roles": [
+        "regionA-read"
+      ]
     }
   ]
 }



[46/50] [abbrv] incubator-geode git commit: GEODE-1397: Adding new help text from create async-event-queue

Posted by kl...@apache.org.
GEODE-1397: Adding new help text from create async-event-queue


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

Branch: refs/heads/feature/GEODE-835
Commit: 5ebd3ff28d3ed2e3a06a8a753277f717cf384647
Parents: cafda19
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 18 14:30:40 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu May 19 19:34:51 2016 -0700

----------------------------------------------------------------------
 .../cli/commands/golden-help-offline.properties         | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5ebd3ff2/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 a9dc64c..74db393 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,9 +540,10 @@ 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)?] [--max-queue-memory=value] [--dispatcher-threads=value]\n\
-\ \ \ \ [--order-policy=value] [--gateway-event-filter=value(,value)*]\n\
-\ \ \ \ [--gateway-event-substitution-filter=value] [--listener-param=value(,value)*]\n\
+\ \ \ \ [--disk-synchronous(=value)?] [--ignore-eviction-expiration(=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\
 PARAMETERS\n\
 \ \ \ \ id\n\
 \ \ \ \ \ \ \ \ ID of the queue to be created.\n\
@@ -582,6 +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\
+\ \ \ \ \ \ \ \ Required: false\n\
+\ \ \ \ \ \ \ \ Default (if the parameter is specified without value): true\n\
+\ \ \ \ \ \ \ \ Default (if the parameter is not specified): true\n\
 \ \ \ \ max-queue-memory\n\
 \ \ \ \ \ \ \ \ Maximum amount of memory, in megabytes, that the queue can consume before overflowing to\n\
 \ \ \ \ \ \ \ \ disk.\n\


[45/50] [abbrv] incubator-geode git commit: GEODE-1397: Fixing testOfflineHelp

Posted by kl...@apache.org.
GEODE-1397: Fixing testOfflineHelp


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

Branch: refs/heads/feature/GEODE-835
Commit: cafda19311588efd43f56736aaab26a66365f58d
Parents: dd8ea42
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 18 14:29:43 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu May 19 19:34:51 2016 -0700

----------------------------------------------------------------------
 .../commands/HelpCommandsIntegrationTest.java   | 20 ++++++++++++++------
 1 file changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cafda193/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
index 5d43a0e..2c048fc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
@@ -16,7 +16,9 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.management.internal.cli.CommandManager;
 import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
@@ -27,8 +29,10 @@ import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.ClassRule;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.contrib.java.lang.system.ProvideSystemProperty;
 import org.junit.experimental.categories.Category;
 
 import java.util.Map;
@@ -39,12 +43,15 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
 @Category(IntegrationTest.class)
-public class HelpCommandsIntegrationTest extends JUnit4DistributedTestCase {
+public class HelpCommandsIntegrationTest {
 
   private int jmxPort;
 
   private Gfsh gfsh;
 
+  @ClassRule
+  public static final ProvideSystemProperty isGfsh = new ProvideSystemProperty("gfsh", "true");
+
   @Before
   public void setup() throws Exception {
     jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
@@ -53,19 +60,20 @@ public class HelpCommandsIntegrationTest extends JUnit4DistributedTestCase {
     localProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
     localProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
     localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
-    getSystem(localProps);
+
+    new CacheFactory(localProps).create();
 
     gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
   }
 
   @After
   public void teardown() {
-    disconnectAllFromDS();
-
-    gfsh.executeCommand("disconnect");
+    InternalDistributedSystem ids = InternalDistributedSystem.getConnectedInstance();
+    if (ids != null) {
+      ids.disconnect();
+    }
   }
 
-  @Ignore("Disconnect command doesn't appear to be working")
   @Test
   public void testOfflineHelp() throws Exception {
     Properties helpProps = new Properties();


[20/50] [abbrv] incubator-geode git commit: GEODE-1401: handle NOT_AVAILABLE in callers of getRawOldValue

Posted by kl...@apache.org.
GEODE-1401: handle NOT_AVAILABLE in callers of getRawOldValue


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

Branch: refs/heads/feature/GEODE-835
Commit: ab417e32deccb4e398d21e182bb738af9ef65e24
Parents: 11da98d
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon May 16 16:57:56 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue May 17 09:58:09 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/BucketRegionQueue.java         | 10 ++--------
 .../gemstone/gemfire/internal/cache/EntryEventImpl.java   |  2 +-
 .../gemfire/internal/cache/RemoteDestroyMessage.java      |  2 +-
 3 files changed, 4 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab417e32/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
index 0243cde..a3d9d97 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegionQueue.java
@@ -269,10 +269,7 @@ public class BucketRegionQueue extends AbstractBucketRegionQueue {
         requireOldValue, lastModified, overwriteDestroyed);
 
     if (success) {
-      Object ov = event.getRawOldValue();
-      if (ov instanceof GatewaySenderEventImpl) {
-        ((GatewaySenderEventImpl) ov).release();
-      }
+      GatewaySenderEventImpl.release(event.getRawOldValue());
 
       if (getPartitionedRegion().getColocatedWith() == null) {
         return success;
@@ -370,10 +367,7 @@ public class BucketRegionQueue extends AbstractBucketRegionQueue {
     }
     super.basicDestroy(event, cacheWrite, expectedOldValue);
 
-    Object rov = event.getRawOldValue();
-    if (rov instanceof GatewaySenderEventImpl) {
-      ((GatewaySenderEventImpl) rov).release();
-    }
+    GatewaySenderEventImpl.release(event.getRawOldValue());
     // Primary buckets should already remove the key while peeking
     if (!this.getBucketAdvisor().isPrimary()) {
       if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab417e32/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
index e241622..b833ecb 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -1407,7 +1407,7 @@ public class EntryEventImpl
         importer.importOldObject(cdV, true);
       }
     } else {
-      importer.importOldObject(ov, true);
+      importer.importOldObject(AbstractRegion.handleNotAvailable(ov), true);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ab417e32/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
index bb97d9c..f7109bb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
@@ -232,7 +232,7 @@ public class RemoteDestroyMessage extends RemoteOperationMessageWithDirectReply
           setOldValBytes((byte[]) old);
         } else {
           this.oldValueIsSerialized = true;
-          setOldValObj(old);
+          setOldValObj(AbstractRegion.handleNotAvailable(old));
         }
       }
     }


[47/50] [abbrv] incubator-geode git commit: GEODE-1397: Add additional DS properties to override autogenerated ones

Posted by kl...@apache.org.
GEODE-1397: Add additional DS properties to override autogenerated ones


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

Branch: refs/heads/feature/GEODE-835
Commit: 2dd3d9065e8cfb942c88ee24b28bffbfcc1e0357
Parents: 5ebd3ff
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri May 20 07:51:00 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri May 20 07:51:00 2016 -0700

----------------------------------------------------------------------
 .../internal/cli/commands/HelpCommandsIntegrationTest.java         | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2dd3d906/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
index 2c048fc..0707766 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
@@ -57,6 +57,8 @@ public class HelpCommandsIntegrationTest {
     jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
     Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     localProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
     localProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
     localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));


[41/50] [abbrv] incubator-geode git commit: GEODE-11: Adding a test of using a KeywordAnalyzer

Posted by kl...@apache.org.
GEODE-11: Adding a test of using a KeywordAnalyzer

Testing that a KeywordAnalyzer does not tokenize fields.


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

Branch: refs/heads/feature/GEODE-835
Commit: 8d1fa0c32de3369e39c05bc9a48c64149df09a98
Parents: 9d23a69
Author: Dan Smith <up...@apache.org>
Authored: Wed May 18 15:27:35 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu May 19 17:02:05 2016 -0700

----------------------------------------------------------------------
 .../lucene/LuceneQueriesIntegrationTest.java    | 44 ++++++++++++++++++++
 .../cache/lucene/test/LuceneTestUtilities.java  | 23 +++++++++-
 .../gemfire/cache/lucene/test/TestObject.java   | 29 ++++++++++++-
 3 files changed, 93 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1fa0c3/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index 2262339..15f5747 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -16,15 +16,25 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.verifyQueryKeys;
 import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.*;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
 import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -41,6 +51,33 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   protected static final String REGION_NAME = "index";
 
   @Test()
+  public void shouldNotTokenizeWordsWithKeywordAnalyzer() throws ParseException {
+    Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
+    fields.put("field1", new StandardAnalyzer());
+    fields.put("field2", new KeywordAnalyzer());
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, fields);
+    Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
+      .create(REGION_NAME);
+    final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    //Put two values with some of the same tokens
+    String value1 = "one three";
+    region.put("A", new TestObject(value1, value1));
+    String value2 = "one two three";
+    region.put("B", new TestObject(value2, value2));
+
+    index.waitUntilFlushed(60000);
+
+    //Using the standard analyzer, this query will match both results
+    verifyQuery("field1:\"one three\"", "A", "B");
+
+    //Using the keyword analyzer, this query will only match the entry that exactly matches
+    verifyQuery("field2:\"one three\"", "A");
+
+
+  }
+
+  @Test()
   public void throwFunctionExceptionWhenGivenBadQuery() {
     LuceneService luceneService = LuceneServiceProvider.get(cache);
     luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
@@ -65,5 +102,12 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
 
   }
 
+  private void verifyQuery(String query, String ... expectedKeys) throws ParseException {
+    final LuceneQuery<Object, Object> queryWithStandardAnalyzer = luceneService.createLuceneQueryFactory().create(
+      INDEX_NAME, REGION_NAME, query);
+
+    verifyQueryKeys(queryWithStandardAnalyzer, expectedKeys);
+  }
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1fa0c3/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 61355c4..7a3ef04 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -18,13 +18,20 @@
  */
 package com.gemstone.gemfire.cache.lucene.test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.function.Consumer;
+import java.util.stream.Collectors;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
@@ -61,4 +68,18 @@ public class LuceneTestUtilities {
     boolean flushed = index.waitUntilFlushed(60000);
     assertTrue(flushed);
   }
+
+  /**
+   * Verify that a query returns the expected list of keys. Ordering is ignored.
+   */
+  public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) {
+    Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
+    Set<K> actualKeySet = new HashSet<>(Arrays.asList(expectedKeys));
+    final LuceneQueryResults<K, Object> results = query.search();
+    while(results.hasNextPage()) {
+      results.getNextPage().stream()
+        .forEach(struct -> actualKeySet.add(struct.getKey()));
+    }
+    assertEquals(expectedKeySet, actualKeySet);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1fa0c3/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
index 3287b9f..a2b249e 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/TestObject.java
@@ -22,6 +22,31 @@ import java.io.Serializable;
 
 public class TestObject implements Serializable {
 
-  String field1 = "hello world";
-  String field2 = "this is a field";
+  private String field1 = "hello world";
+  private String field2 = "this is a field";
+
+  public TestObject() {
+
+  }
+
+  public TestObject(final String field1, final String field2) {
+    this.field1 = field1;
+    this.field2 = field2;
+  }
+
+  public String getField1() {
+    return field1;
+  }
+
+  public void setField1(final String field1) {
+    this.field1 = field1;
+  }
+
+  public String getField2() {
+    return field2;
+  }
+
+  public void setField2(final String field2) {
+    this.field2 = field2;
+  }
 }


[12/50] [abbrv] incubator-geode git commit: GEODE-1393 locator returns incorrect server information when starting up

Posted by kl...@apache.org.
GEODE-1393 locator returns incorrect server information when starting up

When a locator auto-reconnects its ServerLocator needs to initialize its
ControllerAdvisor so that it has server information to give to clients.
The ServerLocator was creating a new ControllerAdvisor but didn't ask it
to perform a handshake to fill in its profiles.

ReconnectDUnitTest had an existing testReconnectWithQuorum test that
wasn't doing what it was supposed to.  I've removed the TODO from that
test and modified it to force-disconnect the tests Locator.  The
locator must restart its TcpServer component before it can start
a DistributedSystem, so this exercises the path in
InternalLocator.attemptReconnect() that boots the TcpServer prior to
connecting the DistributedSystem.  After the DistributedSystem
finishes reconnecting the ServerLocator's distribution advisor
should have been initialized by performing the handshake.


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

Branch: refs/heads/feature/GEODE-835
Commit: 6523c97c92f607746d80b11c7cb5315b1137f5a2
Parents: 92805bb
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Mon May 16 08:05:00 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Mon May 16 08:06:49 2016 -0700

----------------------------------------------------------------------
 .../distributed/internal/InternalLocator.java   |  1 +
 .../distributed/internal/LocatorStats.java      | 31 -----------
 .../distributed/internal/ServerLocator.java     | 44 +++++++--------
 .../gemfire/cache30/ReconnectDUnitTest.java     | 57 ++++++++++----------
 .../internal/ServerLocatorJUnitTest.java        |  0
 5 files changed, 53 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6523c97c/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
index 7effa3d..7ad57ad 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalLocator.java
@@ -1350,6 +1350,7 @@ public class InternalLocator extends Locator implements ConnectListener {
         return response;
       }
     }
+
     private JmxManagerLocatorResponse findJmxManager(JmxManagerLocatorRequest request) {
       JmxManagerLocatorResponse result = null;
       // NYI

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6523c97c/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
old mode 100644
new mode 100755
index d42a2b4..1140b1f
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/LocatorStats.java
@@ -117,13 +117,6 @@ public class LocatorStats {
   }
   
   
-  /**
-   * Used by tests to create an instance given its already existings stats.
-   */
-  public LocatorStats(Statistics stats) {
-    this._stats = stats;
-  }
-
   public final void setServerCount(int sc) {
     if(this._stats==null) {
       this.endpoints_known.set(sc);
@@ -140,14 +133,6 @@ public class LocatorStats {
     }
   }
   
-  public final void incLocatorRequests() {
-    if(this._stats==null) {
-      this.requests_to_locator.incrementAndGet();
-    } else {
-      this._stats.incLong(_REQUESTS_TO_LOCATOR, 1);
-    }
-  }  
-  
   public final void endLocatorRequest(long startTime) {
     long took = DistributionStats.getStatTime()-startTime;
     if(this._stats==null) {
@@ -180,14 +165,6 @@ public class LocatorStats {
   
   
   
-  public final void incLocatorResponses() {
-    if(this._stats==null) {
-      this.responses_from_locator.incrementAndGet();
-    } else {
-      this._stats.incLong(_RESPONSES_FROM_LOCATOR, 1);
-    }
-  }  
-  
   public final void setLocatorRequests(long rl) {
     if(this._stats==null) {
       this.requests_to_locator.set(rl);
@@ -218,14 +195,6 @@ public class LocatorStats {
     } else {
       this._stats.incLong(_SERVER_LOAD_UPDATES, 1);
     }
-  }  
-  
-  public void setRequestInProgress(int threads) {
-    if(this._stats!=null) {
-      this._stats.setInt(_REQUESTS_IN_PROGRESS, threads);
-    } else {
-      requestsInProgress.set(threads);
-    }
   }
   
   public void incRequestInProgress(int threads) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6523c97c/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
old mode 100644
new mode 100755
index e535b97..b37a50b
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
@@ -24,12 +24,12 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelCriterion;
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.cache.client.internal.locator.GetAllServersRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.GetAllServersResponse;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorListRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorListResponse;
-import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.LocatorStatusResponse;
 import com.gemstone.gemfire.cache.client.internal.locator.QueueConnectionRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.QueueConnectionResponse;
@@ -176,34 +175,34 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
       logger.debug("ServerLocator: Received request {}", request);
     }
 
-    Object response;
+    if ( ! (request instanceof ServerLocationRequest) ) {
+      throw new InternalGemFireException("Expected ServerLocationRequest, got " + request.getClass());
+    }
 
-    if (request instanceof ServerLocationRequest) {
-      if (request instanceof LocatorStatusRequest) {
+    Object response;
+    int id = ((DataSerializableFixedID)request).getDSFID();
+    switch (id) {
+      case DataSerializableFixedID.LOCATOR_STATUS_REQUEST:
         response = new LocatorStatusResponse()
           .initialize(this.port, this.hostName, this.logFile, this.memberName);
-      }
-      else if (request instanceof LocatorListRequest) {
+        break;
+      case DataSerializableFixedID.LOCATOR_LIST_REQUEST:
         response = getLocatorListResponse((LocatorListRequest) request);
-      }
-      else if (request instanceof ClientReplacementRequest) {
+        break;
+      case DataSerializableFixedID.CLIENT_REPLACEMENT_REQUEST:
         response = pickReplacementServer((ClientReplacementRequest) request);
-      }
-      else if (request instanceof GetAllServersRequest) {
+        break;
+      case DataSerializableFixedID.GET_ALL_SERVERS_REQUEST:
         response = pickAllServers((GetAllServersRequest) request);
-      }
-      else if (request instanceof ClientConnectionRequest) {
+        break;
+      case DataSerializableFixedID.CLIENT_CONNECTION_REQUEST:
         response = pickServer((ClientConnectionRequest) request);
-      }
-      else if (request instanceof QueueConnectionRequest) {
+        break;
+      case DataSerializableFixedID.QUEUE_CONNECTION_REQUEST:
         response = pickQueueServers((QueueConnectionRequest) request);
-      }
-      else {
+        break;
+      default:
         throw new InternalGemFireException("Unknown ServerLocationRequest: " + request.getClass());
-      }
-    }
-    else {
-      throw new InternalGemFireException("Expected ServerLocationRequest, got " + request.getClass());
     }
 
     if(logger.isDebugEnabled()) {
@@ -290,6 +289,9 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
       this.loadSnapshot = new LocatorLoadSnapshot();
       this.ds = (InternalDistributedSystem)ds;
       this.advisor = ControllerAdvisor.createControllerAdvisor(this); // escapes constructor but allows field to be final
+      if (ds.isConnected()) {
+        this.advisor.handshake();  // GEODE-1393: need to get server information during restart
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6523c97c/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
index ca2c17b..6c63def 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
@@ -27,6 +27,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem.ReconnectListener;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.ServerLocator;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
@@ -78,6 +79,7 @@ public class ReconnectDUnitTest extends CacheTestCase
           locatorPort = locPort;
           Properties props = getDistributedSystemProperties();
           locator = Locator.startLocatorAndDS(locatorPort, new File(""), props);
+          ReconnectDUnitTest.savedSystem = InternalDistributedSystem.getConnectedInstance();
           IgnoredException.addIgnoredException("com.gemstone.gemfire.ForcedDisconnectException||Possible loss of quorum");
 //          MembershipManagerHelper.getMembershipManager(InternalDistributedSystem.getConnectedInstance()).setDebugJGroups(true);
         } catch (IOException e) {
@@ -163,10 +165,6 @@ public class ReconnectDUnitTest extends CacheTestCase
     return factory.create();
   }
 
-  /*
-  TODO this test is not actually using quorum checks.  To do that it needs to
-  have the locator disconnect & reconnect
-   */
 
   public void testReconnectWithQuorum() throws Exception {
     // quorum check fails, then succeeds
@@ -174,7 +172,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
+    VM locatorVm = host.getVM(locatorVMNumber);
     
     final int locPort = locatorPort;
 
@@ -210,33 +208,23 @@ public class ReconnectDUnitTest extends CacheTestCase
       }
     };
     
-    System.out.println("creating caches in vm0, vm1 and vm2");
+    System.out.println("creating caches in vm0 and vm1");
     vm0.invoke(create);
     vm1.invoke(create);
-    vm2.invoke(create);
-    
+
     // view is [locator(3), vm0(15), vm1(10), vm2(10)]
     
-    /* now we want to cause vm0 and vm1 to force-disconnect.  This may cause the other
-     * non-locator member to also disconnect, depending on the timing
+    /* now we want to kick out the locator and observe that it reconnects
+     * using its rebooted location service
      */
-    System.out.println("disconnecting vm0");
-    forceDisconnect(vm0);
-    Wait.pause(10000);
-    System.out.println("disconnecting vm1");
-    forceDisconnect(vm1);
+    System.out.println("disconnecting locator");
+    forceDisconnect(locatorVm);
+    waitForReconnect(locatorVm);
+
+    // if the locator reconnected it did so with its own location
+    // service since it doesn't know about any other locators
+    ensureLocationServiceRunning(locatorVm);
 
-    /* now we wait for them to auto-reconnect */
-    try {
-      System.out.println("waiting for vm0 to reconnect");
-      waitForReconnect(vm0);
-      System.out.println("waiting for vm1 to reconnect");
-      waitForReconnect(vm1);
-      System.out.println("done reconnecting vm0 and vm1");
-    } catch (Exception e) {
-      ThreadUtils.dumpAllStacks();
-      throw e;
-    }
   }
   
   public void testReconnectOnForcedDisconnect() throws Exception  {
@@ -418,6 +406,19 @@ public class ReconnectDUnitTest extends CacheTestCase
       }
     });
   }
+
+  /** this will throw an exception if location services aren't running */
+  private void ensureLocationServiceRunning(VM vm) {
+    vm.invoke(new SerializableRunnable("ensureLocationServiceRunning") {
+      public void run() {
+        InternalLocator intloc = (InternalLocator)locator;
+        ServerLocator serverLocator = intloc.getServerLocatorAdvisee();
+        // the initialization flag in the locator's ControllerAdvisor will
+        // be set if a handshake has been performed
+        assertTrue(serverLocator.getDistributionAdvisor().isInitialized());
+      }
+    });
+  }
   
   private DistributedMember waitForReconnect(VM vm) {
     return (DistributedMember)vm.invoke(new SerializableCallable("wait for Reconnect and return ID") {
@@ -456,7 +457,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     VM vm1 = host.getVM(1);
-    VM vm3 = host.getVM(3);
+    VM locatorVm = host.getVM(3);
     DistributedMember dm, newdm;
     
     final int locPort = locatorPort;
@@ -467,7 +468,7 @@ public class ReconnectDUnitTest extends CacheTestCase
     final String xmlFileLoc = (new File(".")).getAbsolutePath();
 
     //This locator was started in setUp.
-    File locatorViewLog = new File(vm3.getWorkingDirectory(), "locator"+locatorPort+"views.log");
+    File locatorViewLog = new File(locatorVm.getWorkingDirectory(), "locator"+locatorPort+"views.log");
     assertTrue("Expected to find " + locatorViewLog.getPath() + " file", locatorViewLog.exists());
     long logSize = locatorViewLog.length();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6523c97c/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/ServerLocatorJUnitTest.java
old mode 100644
new mode 100755


[40/50] [abbrv] incubator-geode git commit: GEODE-1352: Clean up cache in LuceneQueryImplIntegrationTest

Posted by kl...@apache.org.
GEODE-1352: Clean up cache in LuceneQueryImplIntegrationTest

Making this test case extend the LuceneIntegrationTest to clean up the
cache after the test runs.


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

Branch: refs/heads/feature/GEODE-835
Commit: f272cf0051e599d18f0ba867b5c3fa5d67703502
Parents: 4a97daf
Author: Dan Smith <up...@apache.org>
Authored: Wed May 18 17:21:25 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu May 19 17:02:05 2016 -0700

----------------------------------------------------------------------
 .../internal/LuceneQueryImplIntegrationTest.java      | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f272cf00/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
index 90ab022..262efaa 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
@@ -36,6 +36,7 @@ import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.execute.FunctionAdapter;
 import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
@@ -46,25 +47,22 @@ import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollecto
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 @Category(IntegrationTest.class)
-public class LuceneQueryImplIntegrationTest {
+public class LuceneQueryImplIntegrationTest extends LuceneIntegrationTest {
   private static int LIMIT = 123;
-  private Cache cache;
   private Region<Object, Object> region;
 
   @Before
-  public void createCache() {
-    cache = new CacheFactory().set("mcast-port", "0").create();
-    region = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).create("region");
+  public void createRegion() {
+    region = cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
   }
 
   @After
-  public void removeCache() {
+  public void removeFunction() {
     FunctionService.unregisterFunction(LuceneFunction.ID);
-    cache.close();
   }
 
   @Test
-  public void test() {
+  public void shouldInvokeLuceneFunctionWithCorrectArguments() {
     // Register a fake function to observe the function invocation
     FunctionService.unregisterFunction(LuceneFunction.ID);
     TestLuceneFunction function = new TestLuceneFunction();


[39/50] [abbrv] incubator-geode git commit: GEODE-1391 JsonFormatter javadocs need to be cleaned up

Posted by kl...@apache.org.
GEODE-1391 JsonFormatter javadocs need to be cleaned up

I've made another pass at JSONFormatter javadocs, putting the JSON <-> Java
types in a table and correcting a few grammatical errors.  I also found that
the class was being referred to as PdxFormatter in a lot of places and that
its test was also using this name instead of JSONFormatter.


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

Branch: refs/heads/feature/GEODE-835
Commit: 9d23a69506a0ff531831147c4c704a3553aa6d9f
Parents: 14a548f
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu May 19 14:11:50 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu May 19 14:20:14 2016 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/pdx/JSONFormatter.java |   69 +-
 .../gemfire/pdx/JSONFormatterJUnitTest.java     |  208 ++++
 .../pdx/JSONPdxClientServerDUnitTest.java       |   10 +-
 .../pdx/PdxFormatterPutGetJUnitTest.java        |  208 ----
 .../gemfire/pdx/TestObjectForJSONFormatter.java | 1000 +++++++++++++++++
 .../gemfire/pdx/TestObjectForPdxFormatter.java  | 1003 ------------------
 6 files changed, 1251 insertions(+), 1247 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d23a695/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java b/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
index b9b6329..61f5643 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/pdx/JSONFormatter.java
@@ -32,27 +32,34 @@ import com.gemstone.gemfire.pdx.internal.json.PdxToJSON;
 
 
 /**
- * PdxFormatter class has static method {@link JSONFormatter#fromJSON(String)} to convert JSON 
- * document into a {@link PdxInstance} and static method {@link JSONFormatter#toJSON(PdxInstance)} 
- * to convert back {@link PdxInstance} into a JSON Document.
- * 
- * Using this, application can put PdxInstance(a converted JSON document) in a geode cache. 
- * Application can define indexes on PdxInstance and then query those using OQL. Query will 
- * return the PdxInstances as a result, that needs to convert back into JSON document. 
- * 
- * This uses Jackson parser to parse the json document. Parser treats values in json document as 
- * number(byte, short, int, long..), string, array, object, 'true', 'false' or 'null'. Which
- * further treated as following java types in PdxInstance
- * 
- * JSON object is converted into {@link PdxInstance}
- * JSON arrays is converted into {@link java.util.LinkedList}
- * JSON BigDecimal is converted into {@link BigDecimal}
- * JSON BigInterger is converted into {@link BigInteger}
- * JSON Double is converted into java primitive double
- * JSON float is converted into java primitive float 
- * JSON boolean is converted into java primitive boolean
- * JSON Integer is converted based on its range to java byte, short or int.
- * JSON null is converted java null object.
+ * <p>
+ * JSONFormatter has a static method {@link JSONFormatter#fromJSON(String)} to convert a JSON
+ * document into a {@link PdxInstance} and a static method {@link JSONFormatter#toJSON(PdxInstance)}
+ * to convert a {@link PdxInstance} into a JSON Document.
+ * </p>
+ * <p>
+ * Using these methods an applications may convert a JSON document into a PdxInstance for storing in the cache.
+ * Indexes can then be defined on the PdxInstances so that queries can be performed using OQL. Queries will
+ * return PdxInstances and these can be turned back back into JSON documents using JSONFormatter.
+ * </p>
+ * <p>
+ * JSONFormatter treats values in a json document as
+ * number(byte, short, int, long..), string, array, object, 'true', 'false' or 'null'. These correspond
+ * to the following java types:
+ * </p>
+ *
+ * <table>
+ *   <th>JSON</th><th>Java</th>
+ * <tr> <td>object</td>      <td>{@link PdxInstance}</td> </tr>
+ * <tr> <td>arrays</td>      <td>{@link java.util.LinkedList}</td> </tr>
+ * <tr> <td>BigDecimal</td>  <td>{@link BigDecimal}</td> </tr>
+ * <tr> <td>BigInterger</td> <td>{@link BigInteger}</td> </tr>
+ * <tr> <td>Double</td>      <td>double</td> </tr>
+ * <tr> <td>float</td>       <td>float</td> </tr>
+ * <tr> <td>boolean</td>     <td>boolean</td> </tr>
+ * <tr> <td>Integer</td>     <td>byte, short or int</td> </tr>
+ * <tr> <td>null</td>        <td>null</td> </tr>
+ * </table>
  */
 
 public class JSONFormatter {
@@ -65,10 +72,10 @@ public class JSONFormatter {
   }
   
   /**
-   * To create PdxInstance from JSON string
+   * Converts a JSON document into a PdxInstance
    * 
    * @return the PdxInstance.
-   * @throws JSONFormatterException if unable to parse JSON doucment
+   * @throws JSONFormatterException if unable to parse the JSON document
    */
   public static PdxInstance fromJSON(String jsonString) {
     JsonParser jp = null;
@@ -86,10 +93,10 @@ public class JSONFormatter {
   }
   
   /**
-   * To create PdxInstance from JSON string
+   * Converts a JSON document into a PdxInstance
    * 
    * @return the PdxInstance.
-   * @throws JSONFormatterException if unable to parse JSON doucment
+   * @throws JSONFormatterException if unable to parse the JSON document
    */
   public static PdxInstance fromJSON(byte[] jsonByteArray) {
     JsonParser jp = null;
@@ -112,32 +119,32 @@ public class JSONFormatter {
   }
   
   /**
-   *To create JSON string from PdxInstance
+   * Converts a PdxInstance into a JSON document
    * 
    * @return the JSON string.
-   * @throws JSONFormatterException if unable to create JSON doucment from PdxInstance
+   * @throws JSONFormatterException if unable to create the JSON document
    */
   public static String toJSON(PdxInstance pdxInstance) {
     try {
       PdxToJSON pj = new PdxToJSON(pdxInstance);
       return pj.getJSON();
     } catch (Exception e) {
-      throw new JSONFormatterException("Could not create JSON document from PdxInstance ", e);
+      throw new JSONFormatterException("Could not create JSON document from PdxInstance", e);
     }    
   }
   
   /**
-   *To create JSON byte array from PdxInstance
+   * Converts a PdxInstance into a JSON document in byte-array form
    * 
    * @return the JSON byte array.
-   * @throws JSONFormatterException if unable to create JSON doucment from PdxInstance
+   * @throws JSONFormatterException if unable to create the JSON document
    */
   public static byte[] toJSONByteArray(PdxInstance pdxInstance) {
     try {
       PdxToJSON pj = new PdxToJSON(pdxInstance);
       return pj.getJSONByteArray();
     } catch (Exception e) {
-      throw new JSONFormatterException("Could not create JSON document from PdxInstance ", e);
+      throw new JSONFormatterException("Could not create JSON document from PdxInstance", e);
     }    
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d23a695/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONFormatterJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONFormatterJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONFormatterJUnitTest.java
new file mode 100755
index 0000000..458c020
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONFormatterJUnitTest.java
@@ -0,0 +1,208 @@
+/*
+ * 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.pdx;
+
+import static org.junit.Assert.fail;
+
+import java.text.SimpleDateFormat;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class JSONFormatterJUnitTest {
+
+  private GemFireCacheImpl c;
+  private final String PRIMITIVE_KV_STORE_REGION = "primitiveKVStore";
+    
+  public JSONFormatterJUnitTest() {
+    super();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    this.c = (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").setPdxReadSerialized(true).create();
+    
+    //start cache-server
+    CacheServer server = c.addCacheServer();
+    final int serverPort = 40405;
+    server.setPort(serverPort);
+    server.start();
+    
+    // Create region, primitiveKVStore
+    final AttributesFactory<Object, Object> af1 = new AttributesFactory<Object, Object>();
+    af1.setDataPolicy(DataPolicy.PARTITION);
+    final RegionAttributes<Object, Object> rAttributes = af1.create();
+    c.createRegion(PRIMITIVE_KV_STORE_REGION, rAttributes);
+  }
+
+  @After
+  public void tearDown() {
+    //shutdown and clean up the manager node.
+    this.c.close();
+  }
+  
+  private void ValidatePdxInstanceToJsonConversion(){
+    
+    Cache c = CacheFactory.getAnyInstance();
+    Region region = c.getRegion("primitiveKVStore");
+    
+    TestObjectForJSONFormatter actualTestObject = new TestObjectForJSONFormatter();
+    actualTestObject.defaultInitialization();
+
+    //Testcase-1: PdxInstance to Json conversion
+    //put Object and getvalue as Pdxinstance
+    region.put("201", actualTestObject);
+    Object receivedObject = region.get("201");
+
+    //PdxInstance->Json conversion
+    if(receivedObject instanceof PdxInstance){
+      PdxInstance pi = (PdxInstance)receivedObject;
+      String json = JSONFormatter.toJSON(pi);
+
+      verifyJsonWithJavaObject(json, actualTestObject);
+    }else {
+      fail("receivedObject is expected to be of type PdxInstance");
+    }
+
+  }
+
+  //Testcase-2: validate Json->PdxInstance-->Java conversion
+  private void verifyJsonToPdxInstanceConversion(){
+    TestObjectForJSONFormatter expectedTestObject = new TestObjectForJSONFormatter();
+    expectedTestObject.defaultInitialization();
+    Cache c = CacheFactory.getAnyInstance();
+    Region region = c.getRegion("primitiveKVStore");
+
+    //1.gets pdxInstance using R.put() and R.get()
+    region.put("501", expectedTestObject);
+    Object receivedObject = region.get("501");
+    if(receivedObject instanceof PdxInstance){
+      PdxInstance expectedPI = (PdxInstance)receivedObject;
+
+      //2. Get the JSON string from actualTestObject using jackson ObjectMapper.
+      ObjectMapper objectMapper = new ObjectMapper();
+      objectMapper.setDateFormat(new SimpleDateFormat("MM/dd/yyyy"));
+      objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+      String json;
+      try {
+        json = objectMapper.writeValueAsString(expectedTestObject);
+        String jsonWithClassType = expectedTestObject.addClassTypeToJson(json);
+
+        //3. Get PdxInstance from the Json String and Validate pi.getObject() API.
+        PdxInstance actualPI = JSONFormatter.fromJSON(jsonWithClassType);
+        //Note: expectedPI will contains those fields that are part of toData()
+        //      expectedPI.className = "com.gemstone.gemfire.pdx.TestObjectForJSONFormatter"
+        //      actualPI will contains all the fields that are member of the class.
+        //      actualPI..className = __GEMFIRE_JSON
+        //      and hence actualPI.equals(expectedPI) will returns false.
+
+        Object actualTestObject = actualPI.getObject();
+        if(actualTestObject instanceof TestObjectForJSONFormatter){
+          boolean isObjectEqual = actualTestObject.equals(expectedTestObject);
+          Assert.assertTrue(isObjectEqual, "actualTestObject and expectedTestObject should be equal");
+        }else {
+          fail("actualTestObject is expected to be of type PdxInstance");
+        }
+      } catch (JsonProcessingException e1) {
+        fail("JsonProcessingException occurred:" + e1.getMessage());
+      } catch (JSONException e) {
+        fail("JSONException occurred:" + e.getMessage());
+      }
+    }else {
+      fail("receivedObject is expected to be of type PdxInstance");
+    }
+  }
+
+  private void verifyJsonWithJavaObject (String json, TestObjectForJSONFormatter testObject) {
+    try {
+      JSONObject jsonObject = new JSONObject(json);
+
+      //Testcase-1: Validate json string against the pdxInstance.
+      //validation for primitive types
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Int type values are not matched",
+          testObject.getP_int(), jsonObject.getInt(testObject.getP_intFN()));
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: long type values are not matched",
+          testObject.getP_long(), jsonObject.getLong(testObject.getP_longFN()));
+
+      //validation for wrapper types
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Boolean type values are not matched",
+          testObject.getW_bool().booleanValue(), jsonObject.getBoolean(testObject.getW_boolFN()));
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Float type values are not matched",
+          testObject.getW_double().doubleValue(), jsonObject.getDouble(testObject.getW_doubleFN()));
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: bigDec type values are not matched",
+          testObject.getW_bigDec().longValue(), jsonObject.getLong(testObject.getW_bigDecFN()));
+
+      //vlidation for array types
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Byte[] type values are not matched",
+          (int)testObject.getW_byteArray()[1], jsonObject.getJSONArray(testObject.getW_byteArrayFN()).getInt(1));
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Double[] type values are not matched",
+          testObject.getW_doubleArray()[0], jsonObject.getJSONArray(testObject.getW_doubleArrayFN()).getDouble(0));
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: String[] type values are not matched",
+          testObject.getW_strArray()[2], jsonObject.getJSONArray(testObject.getW_strArrayFN()).getString(2));
+
+      //validation for collection types
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: list type values are not matched",
+          testObject.getC_list().get(0),
+          jsonObject.getJSONArray(testObject.getC_listFN()).getString(0));
+
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: stack type values are not matched",
+          testObject.getC_stack().get(2),
+          jsonObject.getJSONArray(testObject.getC_stackFN()).getString(2));
+
+      //validation for Map
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Map type values are not matched",
+          testObject.getM_empByCity().get("Ahmedabad").get(0).getFname(),
+          jsonObject.getJSONObject(testObject.getM_empByCityFN()).getJSONArray("Ahmedabad").getJSONObject(0).getString("fname"));
+
+      //validation Enum
+      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Enum type values are not matched",
+          testObject.getDay().toString(),
+          jsonObject.getString(testObject.getDayFN()));
+
+    } catch (JSONException e) {
+      fail("Error in VerifyPdxInstanceToJson, Malformed json, can not create JSONArray from it");
+    }
+  }
+
+  @Test
+  public void testJSONFormatterAPIs() {
+    ValidatePdxInstanceToJsonConversion();
+    verifyJsonToPdxInstanceConversion();
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d23a695/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
index 55aa5aa..d76b5bd 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/JSONPdxClientServerDUnitTest.java
@@ -141,7 +141,7 @@ public class JSONPdxClientServerDUnitTest extends CacheTestCase {
     Region region = getRootRegion("testSimplePdx");
     
     //Create Object and initialize its members.
-    TestObjectForPdxFormatter testObject = new TestObjectForPdxFormatter();
+    TestObjectForJSONFormatter testObject = new TestObjectForJSONFormatter();
     testObject.defaultInitialization();
 
     // put the object into cache.
@@ -205,7 +205,7 @@ public class JSONPdxClientServerDUnitTest extends CacheTestCase {
     }
     
     //TestCase-2 : Validate Java-->JSON-->PdxInstance --> Java Mapping
-    TestObjectForPdxFormatter actualTestObject = new TestObjectForPdxFormatter();
+    TestObjectForJSONFormatter actualTestObject = new TestObjectForJSONFormatter();
     actualTestObject.defaultInitialization();
     ObjectMapper objectMapper = new ObjectMapper();
     objectMapper.setDateFormat(new SimpleDateFormat("MM/dd/yyyy"));
@@ -227,13 +227,13 @@ public class JSONPdxClientServerDUnitTest extends CacheTestCase {
         
         //4. get the actualType testObject from the pdxInstance and compare it with actualTestObject
         Object getObj = receivedPdxInstance.getObject();
-        if(getObj instanceof TestObjectForPdxFormatter){
-          TestObjectForPdxFormatter receivedTestObject = (TestObjectForPdxFormatter)getObj;
+        if(getObj instanceof TestObjectForJSONFormatter){
+          TestObjectForJSONFormatter receivedTestObject = (TestObjectForJSONFormatter)getObj;
           
           boolean isEqual = actualTestObject.equals(receivedTestObject);
           Assert.assertTrue(isEqual, "actualTestObject and receivedTestObject should be equal");
         }else {
-          fail("getObj is expected to be an instance of TestObjectForPdxFormatter");
+          fail("getObj is expected to be an instance of TestObjectForJSONFormatter");
         }
       }else {
         fail("receivedObject is expected to be of type PdxInstance");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d23a695/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
deleted file mode 100644
index b1a81b5..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * 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.pdx;
-
-import static org.junit.Assert.fail;
-
-import java.text.SimpleDateFormat;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.DeserializationFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-@Category(IntegrationTest.class)
-public class PdxFormatterPutGetJUnitTest {
-
-  private GemFireCacheImpl c;
-  private final String PRIMITIVE_KV_STORE_REGION = "primitiveKVStore";
-    
-  public PdxFormatterPutGetJUnitTest() {
-    super();
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    this.c = (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").setPdxReadSerialized(true).create();
-    
-    //start cache-server
-    CacheServer server = c.addCacheServer();
-    final int serverPort = 40405;
-    server.setPort(serverPort);
-    server.start();
-    
-    // Create region, primitiveKVStore
-    final AttributesFactory<Object, Object> af1 = new AttributesFactory<Object, Object>();
-    af1.setDataPolicy(DataPolicy.PARTITION);
-    final RegionAttributes<Object, Object> rAttributes = af1.create();
-    c.createRegion(PRIMITIVE_KV_STORE_REGION, rAttributes);
-  }
-
-  @After
-  public void tearDown() {
-    //shutdown and clean up the manager node.
-    this.c.close();
-  }
-  
-  private void ValidatePdxInstanceToJsonConversion(){
-    
-    Cache c = CacheFactory.getAnyInstance();
-    Region region = c.getRegion("primitiveKVStore");
-    
-    TestObjectForPdxFormatter actualTestObject = new TestObjectForPdxFormatter();
-    actualTestObject.defaultInitialization();
-    
-    //Testcase-1: PdxInstance to Json conversion
-    //put Object and getvalue as Pdxinstance
-    region.put("201", actualTestObject);
-    Object receivedObject = region.get("201");
-    
-    //PdxInstance->Json conversion
-    if(receivedObject instanceof PdxInstance){
-      PdxInstance pi = (PdxInstance)receivedObject;
-      String json = JSONFormatter.toJSON(pi);
-      
-      verifyJsonWithJavaObject(json, actualTestObject);
-    }else {
-      fail("receivedObject is expected to be of type PdxInstance");
-    }
-    
-  }
-  
-  //Testcase-2: validate Json->PdxInstance-->Java conversion
-  private void VarifyJsonToPdxInstanceConversion(){
-    TestObjectForPdxFormatter expectedTestObject = new TestObjectForPdxFormatter();
-    expectedTestObject.defaultInitialization();
-    Cache c = CacheFactory.getAnyInstance();
-    Region region = c.getRegion("primitiveKVStore");
-    
-    //1.gets pdxInstance using R.put() and R.get()
-    region.put("501", expectedTestObject);
-    Object receivedObject = region.get("501");
-    if(receivedObject instanceof PdxInstance){
-      PdxInstance expectedPI = (PdxInstance)receivedObject;
-      
-      //2. Get the JSON string from actualTestObject using jackson ObjectMapper.
-      ObjectMapper objectMapper = new ObjectMapper();
-      objectMapper.setDateFormat(new SimpleDateFormat("MM/dd/yyyy"));
-      objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
-      
-      String json;
-      try {
-        json = objectMapper.writeValueAsString(expectedTestObject);
-        String jsonWithClassType = expectedTestObject.addClassTypeToJson(json);
-        
-        //3. Get PdxInstance from the Json String and Validate pi.getObject() API. 
-        PdxInstance actualPI = JSONFormatter.fromJSON(jsonWithClassType);
-        //Note: expectedPI will contains those fields that are part of toData()
-        //      expectedPI.className = "com.gemstone.gemfire.pdx.TestObjectForPdxFormatter"
-        //      actualPI will contains all the fields that are member of the class.
-        //      actualPI..className = __GEMFIRE_JSON
-        //      and hence actualPI.equals(expectedPI) will returns false.
-        
-        Object actualTestObject = actualPI.getObject();
-        if(actualTestObject instanceof TestObjectForPdxFormatter){
-          boolean isObjectEqual = actualTestObject.equals(expectedTestObject);
-          Assert.assertTrue(isObjectEqual, "actualTestObject and expectedTestObject should be equal");
-        }else {
-          fail("actualTestObject is expected to be of type PdxInstance");
-        }
-      } catch (JsonProcessingException e1) {
-        fail("JsonProcessingException occurred:" + e1.getMessage());
-      } catch (JSONException e) {
-        fail("JSONException occurred:" + e.getMessage());
-      }
-    }else {
-      fail("receivedObject is expected to be of type PdxInstance");
-    }
-  }
- 
-  private void verifyJsonWithJavaObject (String json, TestObjectForPdxFormatter testObject) {
-    try {  
-      JSONObject jsonObject = new JSONObject(json);
-      
-      //Testcase-1: Validate json string against the pdxInstance.
-      //validation for primitive types
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Int type values are not matched",
-          testObject.getP_int(), jsonObject.getInt(testObject.getP_intFN()));
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: long type values are not matched",
-          testObject.getP_long(), jsonObject.getLong(testObject.getP_longFN()));
-      
-      //validation for wrapper types
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Boolean type values are not matched",
-          testObject.getW_bool().booleanValue(), jsonObject.getBoolean(testObject.getW_boolFN()));
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Float type values are not matched",
-          testObject.getW_double().doubleValue(), jsonObject.getDouble(testObject.getW_doubleFN()));
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: bigDec type values are not matched",
-          testObject.getW_bigDec().longValue(), jsonObject.getLong(testObject.getW_bigDecFN()));
-      
-      //vlidation for array types
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Byte[] type values are not matched",
-          (int)testObject.getW_byteArray()[1], jsonObject.getJSONArray(testObject.getW_byteArrayFN()).getInt(1));
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Double[] type values are not matched",
-          testObject.getW_doubleArray()[0], jsonObject.getJSONArray(testObject.getW_doubleArrayFN()).getDouble(0));
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: String[] type values are not matched",
-          testObject.getW_strArray()[2], jsonObject.getJSONArray(testObject.getW_strArrayFN()).getString(2));
-      
-      //validation for collection types
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: list type values are not matched", 
-          testObject.getC_list().get(0), 
-          jsonObject.getJSONArray(testObject.getC_listFN()).getString(0));
-      
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: stack type values are not matched", 
-          testObject.getC_stack().get(2), 
-          jsonObject.getJSONArray(testObject.getC_stackFN()).getString(2));
-      
-      //validation for Map
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Map type values are not matched", 
-          testObject.getM_empByCity().get("Ahmedabad").get(0).getFname(), 
-          jsonObject.getJSONObject(testObject.getM_empByCityFN()).getJSONArray("Ahmedabad").getJSONObject(0).getString("fname"));
-      
-      //validation Enum
-      junit.framework.Assert.assertEquals("VerifyPdxInstanceToJson: Enum type values are not matched", 
-          testObject.getDay().toString(), 
-          jsonObject.getString(testObject.getDayFN()));
-      
-    } catch (JSONException e) {
-      fail("Error in VerifyPdxInstanceToJson, Malformed json, can not create JSONArray from it");
-    }
-  }
-  
-  @Test
-  public void testPdxFormatterAPIs() {
-    ValidatePdxInstanceToJsonConversion();
-    VarifyJsonToPdxInstanceConversion();
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d23a695/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForJSONFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForJSONFormatter.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForJSONFormatter.java
new file mode 100755
index 0000000..2347af8
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForJSONFormatter.java
@@ -0,0 +1,1000 @@
+/*
+ * 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.pdx;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.Stack;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+enum Day {
+  Sunday, Monday, Tuesday, Wednesday, Thursday, Friday, Saturday
+}
+
+public class TestObjectForJSONFormatter implements PdxSerializable {
+
+  private boolean p_bool;
+  private byte p_byte;
+  private short p_short;
+  private int p_int;
+  private long p_long;
+  private float p_float;
+  private double p_double;
+  
+  //wrapper
+  private Boolean w_bool;
+  private Byte w_byte;
+  private Short w_short;
+  private Integer w_int;
+  private Long w_long;
+  private BigInteger w_bigInt;
+  private Float w_float;
+  private BigDecimal w_bigDec;
+  private Double w_double;
+  private String w_string;
+  
+  //Primitive_Arrays
+  private boolean[] p_boolArray;
+  private byte[] p_byteArray;
+  private short[] p_shortArray;
+  private int[] p_intArray;
+  private long[] p_longArray;
+  private float[] p_floatArray;
+  private double[] p_doubleArray;
+   
+  //Wrapper_Arrays 
+  private Boolean[] w_boolArray;
+  private Byte[] w_byteArray;
+  private Short[] w_shortArray;
+  private Integer[] w_intArray;
+  private Long[] w_longArray;
+  private BigInteger[] w_bigIntArray;
+  private Float[] w_floatArray;
+  private BigDecimal[] w_bigDecArray;
+  private Double[] w_doubleArray;
+  private String [] w_strArray;
+  
+  //Collection Type: List, Set, Queue, Deque
+  private List<String> c_list;
+  private Set<Object> c_set;
+  private Queue<String> c_queue;
+  private Deque<Integer> c_deque;
+  private Stack<String> c_stack;
+  
+  //Map - Classify Person objects by city
+  Map<String, List<Employee>> m_empByCity;
+  
+  //Enum
+  private Day day;
+
+  private Employee employee;
+  
+  public TestObjectForJSONFormatter(){
+  }
+  
+  public String addClassTypeToJson(String json) throws JSONException {
+    JSONObject jsonObj = new JSONObject(json);
+    jsonObj.put("@type", "com.gemstone.gemfire.pdx.TestObjectForJSONFormatter");
+    return jsonObj.toString();
+  }
+  
+  public void defaultInitialization(){
+    
+    employee = new Employee(1010L, "NilkanthKumar", "Patel");  
+    
+    //Initialize Map type member
+    Employee e1 = new Employee(1L, "Nilkanth", "Patel");
+    Employee e2 = new Employee(2L, "Amey", "Barve");
+    Employee e3 = new Employee(3L, "Shankar", "Hundekar");
+    Employee e4 = new Employee(4L, "Avinash", "Dongre");
+    Employee e5 = new Employee(5L, "supriya", "Patil");
+    Employee e6 = new Employee(6L, "Rajesh", "Kumar");
+    Employee e7 = new Employee(7L, "Vishal", "Rao");
+    Employee e8 = new Employee(8L, "Hitesh", "Khamesara");
+    Employee e9 = new Employee(9L, "Sudhir", "Menon");
+    
+    m_empByCity = new HashMap<String, List<Employee>>();
+    List<Employee> list1 = new ArrayList<Employee>();
+    List<Employee> list2 = new ArrayList<Employee>();
+    List<Employee> list3 = new ArrayList<Employee>();
+    
+    list1.add(e1);
+    list1.add(e2);
+    list1.add(e3);
+    
+    list2.add(e4);
+    list2.add(e5);
+    list2.add(e6);
+    
+    list3.add(e7);
+    list3.add(e8);
+    list3.add(e9);
+    
+    m_empByCity.put("Ahmedabad", list1);
+    m_empByCity.put("mumbai", list2);
+    m_empByCity.put("Pune", list3);
+    
+    
+    //Initialize Collection types members
+    c_list = new ArrayList<String>();
+    c_list.add("Java");
+    c_list.add("scala");
+    c_list.add("closure");
+    
+    c_set = new HashSet<Object>();
+    c_set.add("element 0");
+    c_set.add("element 1");
+    c_set.add("element 2");
+    
+    c_queue = new PriorityQueue<String>(3);
+    c_queue.add("short");
+    c_queue.add("very long indeed");
+    c_queue.add("medium");
+    
+    c_deque = new ArrayDeque<Integer>(4);
+    c_deque.add(15);
+    c_deque.add(30);
+    c_deque.add(20);
+    c_deque.add(18);
+    
+    c_stack = new Stack();
+    c_stack.push( "bat" );
+    c_stack.push( "cat" );
+    c_stack.push( "dog" );
+    
+    //Initialize primitive types members
+    p_bool = true;
+    p_byte = 101;
+    p_short = 32001;
+    p_int = 100001;
+    p_long = 1234567898765432L;
+    p_float = 123.456f;
+    p_double = 98765.12345d; 
+    
+    //Wrapper type member initialization
+    w_bool = new Boolean(false);
+    w_byte = new Byte((byte)11);
+    w_short = new Short((short)101);
+    w_int = new Integer(1001);
+    w_long = new Long(987654321234567L);
+    w_bigInt = new BigInteger("12345678910");
+    w_float = new Float(789.456f);
+    w_bigDec = new BigDecimal(8866333);
+    w_double = new Double(123456.9876d);
+    w_string = new String("Nilkanth Patel");
+  
+    //Initialization for members of type primitive arrays
+    p_boolArray = new boolean[]{ true, false, false};
+    p_byteArray = new byte[]{10, 11, 12};
+    p_shortArray = new short[]{101, 102, 103};
+    p_intArray = new int[]{1001,1002, 1003, 1004, 1005, 1006};
+    p_longArray = new long[]{ 12345678910L, 12345678911L, 12345678912L };
+    p_floatArray = new float[]{ 123.45f, 456.78f, -91011.123f};
+    p_doubleArray = new double[]{1234.5678d, -91011.1213d, 1415.1617d };
+  
+    //Initialization for members of type wrapper arrays
+    w_boolArray = new Boolean[3];
+    w_byteArray = new Byte[3];
+    w_shortArray = new Short[3];
+    w_intArray = new Integer[3];
+    w_longArray = new Long[3];
+    w_floatArray = new Float[3];
+    w_doubleArray = new Double[3];
+    w_strArray = new String[3];
+  
+    for (int i=0; i< 3; i++){
+      w_boolArray[i] = p_boolArray[i];
+      w_byteArray[i] = p_byteArray[i];
+      w_shortArray[i] = p_shortArray[i];
+      w_intArray[i] =  p_intArray[i];
+      w_longArray[i] = p_longArray[i];
+      w_floatArray[i] =  p_floatArray[i];
+      w_doubleArray[i] = p_doubleArray[i];
+    }
+  
+    w_bigIntArray = new BigInteger[] {BigInteger.ZERO, BigInteger.ONE, new BigInteger("12345678910")};
+    w_bigDecArray = new BigDecimal[] {BigDecimal.TEN, new BigDecimal("143.145"), new BigDecimal("10.01")};
+    w_strArray = new String[]{"Nilkanth", "Vishal", "Hitesh"};
+  
+    //Enum type initialization
+    day = Day.Thursday;
+  }
+
+  public TestObjectForJSONFormatter(boolean p_bool, byte p_byte, short p_short,
+      int p_int, long p_long, float p_float, double p_double, Boolean w_bool,
+      Byte w_byte, Short w_short, Integer w_int, Long w_long,
+      BigInteger w_bigInt, Float w_float, BigDecimal w_bigDec,
+      Double w_double, String w_string) {
+    super();
+    this.p_bool = p_bool;
+    this.p_byte = p_byte;
+    this.p_short = p_short;
+    this.p_int = p_int;
+    this.p_long = p_long;
+    this.p_float = p_float;
+    this.p_double = p_double;
+    this.w_bool = w_bool;
+    this.w_byte = w_byte;
+    this.w_short = w_short;
+    this.w_int = w_int;
+    this.w_long = w_long;
+    this.w_bigInt = w_bigInt;
+    this.w_float = w_float;
+    this.w_bigDec = w_bigDec;
+    this.w_double = w_double;
+    this.w_string = w_string;
+  }
+
+  public Employee getEmployee() {
+    return employee;
+  }
+
+  public void setEmployee(Employee employee) {
+    this.employee = employee;
+  }
+  
+  public List<String> getC_list() {
+    return c_list;
+  }
+
+  public void setC_list(List<String> c_list) {
+    this.c_list = c_list;
+  }
+
+  public Set<Object> getC_set() {
+    return c_set;
+  }
+
+  public void setC_set(Set<Object> c_set) {
+    this.c_set = c_set;
+  }
+
+  public Queue<String> getC_queue() {
+    return c_queue;
+  }
+
+  public void setC_queue(Queue<String> c_queue) {
+    this.c_queue = c_queue;
+  }
+
+  public Deque<Integer> getC_deque() {
+    return c_deque;
+  }
+
+  public void setC_deque(Deque<Integer> c_deque) {
+    this.c_deque = c_deque;
+  }
+
+  public Map<String, List<Employee>> getM_empByCity() {
+    return m_empByCity;
+  }
+
+  public void setM_empByCity(Map<String, List<Employee>> m_empByCity) {
+    this.m_empByCity = m_empByCity;
+  }
+
+  public Day getDay() {
+    return day;
+  }
+
+  public void setDay(Day day) {
+    this.day = day;
+  }
+
+  public boolean isP_bool() {
+    return p_bool;
+  }
+        
+  public void setP_bool(boolean p_bool) {
+    this.p_bool = p_bool;
+  }
+  
+  public byte getP_byte() {
+    return p_byte;
+  }
+  
+  public void setP_byte(byte p_byte) {
+    this.p_byte = p_byte;
+  }
+  
+  public short getP_short() {
+    return p_short;
+  }
+  
+  public void setP_short(short p_short) {
+    this.p_short = p_short;
+  }
+  
+  public int getP_int() {
+    return p_int;
+  }
+  
+  public void setP_int(int p_int) {
+    this.p_int = p_int;
+  }
+  
+  public long getP_long() {
+    return p_long;
+  }
+  
+  public void setP_long(long p_long) {
+    this.p_long = p_long;
+  }
+  
+  public float getP_float() {
+    return p_float;
+  }
+  
+  public void setP_float(float p_float) {
+    this.p_float = p_float;
+  }
+  
+  public double getP_double() {
+    return p_double;
+  }
+  
+  public void setP_double(double p_double) {
+    this.p_double = p_double;
+  }
+  
+  public Boolean getW_bool() {
+    return w_bool;
+  }
+  
+  public void setW_bool(Boolean w_bool) {
+    this.w_bool = w_bool;
+  }
+  
+  public Byte getW_byte() {
+    return w_byte;
+  }
+  
+  public void setW_byte(Byte w_byte) {
+    this.w_byte = w_byte;
+  }
+  
+  public Short getW_short() {
+    return w_short;
+  }
+  
+  public void setW_short(Short w_short) {
+    this.w_short = w_short;
+  }
+  
+  public Integer getW_int() {
+    return w_int;
+  }
+  
+  public void setW_int(Integer w_int) {
+    this.w_int = w_int;
+  }
+  
+  public Long getW_long() {
+    return w_long;
+  }
+  
+  public void setW_long(Long w_long) {
+    this.w_long = w_long;
+  }
+  
+  public BigInteger getW_bigInt() {
+    return w_bigInt;
+  }
+  
+  public void setW_bigInt(BigInteger w_bigInt) {
+    this.w_bigInt = w_bigInt;
+  }
+  
+  public Float getW_float() {
+    return w_float;
+  }
+  
+  public void setW_float(Float w_float) {
+    this.w_float = w_float;
+  }
+  
+  public BigDecimal getW_bigDec() {
+    return w_bigDec;
+  }
+  
+  public void setW_bigDec(BigDecimal w_bigDec) {
+    this.w_bigDec = w_bigDec;
+  }
+  
+  public Double getW_double() {
+    return w_double;
+  }
+  
+  public void setW_double(Double w_double) {
+    this.w_double = w_double;
+  }
+  
+  public String getW_string() {
+    return w_string;
+  }
+  
+  public void setW_string(String w_string) {
+    this.w_string = w_string;
+  }
+
+  public boolean[] getP_boolArray() {
+    return p_boolArray;
+  }
+
+  public void setP_boolArray(boolean[] p_boolArray) {
+    this.p_boolArray = p_boolArray;
+  }
+
+  public byte[] getP_byteArray() {
+    return p_byteArray;
+  }
+
+  public void setP_byteArray(byte[] p_byteArray) {
+    this.p_byteArray = p_byteArray;
+  }
+
+  public short[] getP_shortArray() {
+    return p_shortArray;
+  }
+
+  public void setP_shortArray(short[] p_shortArray) {
+    this.p_shortArray = p_shortArray;
+  }
+
+  public int[] getP_intArray() {
+    return p_intArray;
+  }
+
+  public void setP_intArray(int[] p_intArray) {
+    this.p_intArray = p_intArray;
+  }
+
+  public long[] getP_longArray() {
+    return p_longArray;
+  }
+
+  public void setP_longArray(long[] p_longArray) {
+    this.p_longArray = p_longArray;
+  }
+
+  public float[] getP_floatArray() {
+    return p_floatArray;
+  }
+
+  public void setP_floatArray(float[] p_floatArray) {
+    this.p_floatArray = p_floatArray;
+  }
+
+  public double[] getP_doubleArray() {
+    return p_doubleArray;
+  }
+
+  public void setP_doubleArray(double[] p_doubleArray) {
+    this.p_doubleArray = p_doubleArray;
+  }
+
+  public Boolean[] getW_boolArray() {
+    return w_boolArray;
+  }
+
+  public void setW_boolArray(Boolean[] w_boolArray) {
+    this.w_boolArray = w_boolArray;
+  }
+
+  public Byte[] getW_byteArray() {
+    return w_byteArray;
+  }
+
+  public void setW_byteArray(Byte[] w_byteArray) {
+    this.w_byteArray = w_byteArray;
+  }
+
+  public Short[] getW_shortArray() {
+    return w_shortArray;
+  }
+
+  public void setW_shortArray(Short[] w_shortArray) {
+    this.w_shortArray = w_shortArray;
+  }
+
+  public Integer[] getW_intArray() {
+    return w_intArray;
+  }
+
+  public void setW_intArray(Integer[] w_intArray) {
+    this.w_intArray = w_intArray;
+  }
+
+  public Long[] getW_longArray() {
+    return w_longArray;
+  }
+
+  public void setW_longArray(Long[] w_longArray) {
+    this.w_longArray = w_longArray;
+  }
+
+  public BigInteger[] getW_bigIntArray() {
+    return w_bigIntArray;
+  }
+
+  public void setW_bigIntArray(BigInteger[] w_bigIntArray) {
+    this.w_bigIntArray = w_bigIntArray;
+  }
+
+  public Float[] getW_floatArray() {
+    return w_floatArray;
+  }
+
+  public void setW_floatArray(Float[] w_floatArray) {
+    this.w_floatArray = w_floatArray;
+  }
+
+  public BigDecimal[] getW_bigDecArray() {
+    return w_bigDecArray;
+  }
+
+  public void setW_bigDecArray(BigDecimal[] w_bigDecArray) {
+    this.w_bigDecArray = w_bigDecArray;
+  }
+
+  public Double[] getW_doubleArray() {
+    return w_doubleArray;
+  }
+
+  public void setW_doubleArray(Double[] w_doubleArray) {
+    this.w_doubleArray = w_doubleArray;
+  }
+
+  public String[] getW_strArray() {
+    return w_strArray;
+  }
+
+  public void setW_strArray(String[] w_strArray) {
+    this.w_strArray = w_strArray;
+  }
+
+  public Stack<String> getC_stack() {
+    return c_stack;
+  }
+
+  public void setC_stack(Stack<String> c_stack) {
+    this.c_stack = c_stack;
+  }
+
+  // Getters for returning field names
+  public String getP_boolFN() {
+    return "p_bool";
+  }
+
+  public String getP_byteFN() {
+    return "p_byte";
+  }
+
+  public String getP_shortFN() {
+    return "p_short";
+  }
+
+  public String getP_intFN() {
+    return "p_int";
+  }
+
+  public String getP_longFN() {
+    return "p_long";
+  }
+
+  public String getP_floatFn() {
+    return "p_float";
+  }
+
+  public String getP_doubleFN() {
+    return "p_double";
+  }
+
+  public String getW_boolFN() {
+    return "w_bool";
+  }
+
+  public String getW_byteFN() {
+    return "w_byte";
+  }
+
+  public String getW_shortFN() {
+    return "w_short";
+  }
+
+  public String getW_intFN() {
+    return "w_int";
+  }
+
+  public String getW_longFN() {
+    return "w_long";
+  }
+
+  public String getW_bigIntFN() {
+    return "w_bigInt";
+  }
+
+  public String getW_floatFN() {
+    return "w_float";
+  }
+
+  public String getW_bigDecFN() {
+    return "w_bigDec";
+  }
+
+  public String getW_doubleFN() {
+    return "w_double";
+  }
+
+  public String getW_stringFN() {
+    return "w_string";
+  }
+
+  public String getP_boolArrayFN() {
+    return "p_boolArray";
+  }
+
+  public String getP_byteArrayFN() {
+    return "p_byteArray";
+  }
+
+  public String getP_shortArrayFN() {
+    return "p_shortArray";
+  }
+
+  public String getP_intArrayFN() {
+    return "p_intArray";
+  }
+
+  public String getP_longArrayFN() {
+    return "p_longArray";
+  }
+
+  public String getP_floatArrayFN() {
+    return "p_floatArray";
+  }
+
+  public String getP_doubleArrayFN() {
+    return "p_doubleArray";
+  }
+
+  public String getW_boolArrayFN() {
+    return "w_boolArray";
+  }
+
+  public String getW_byteArrayFN() {
+    return "w_byteArray";
+  }
+
+  public String getW_shortArrayFN() {
+    return "w_shortArray";
+  }
+
+  public String getW_intArrayFN() {
+    return "w_intArray";
+  }
+
+  public String getW_longArrayFN() {
+    return "w_longArray";
+  }
+
+  public String getW_bigIntArrayFN() {
+    return "w_bigIntArray";
+  }
+
+  public String getW_floatArrayFN() {
+    return "w_floatArray";
+  }
+
+  public String getW_bigDecArrayFN() {
+    return "w_bigDecArray";
+  }
+
+  public String getW_doubleArrayFN() {
+    return "w_doubleArray";
+  }
+
+  public String getW_strArrayFN() {
+    return "w_strArray";
+  }
+
+  public String getC_listFN() {
+    return "c_list";
+  }
+
+  public String getC_setFN() {
+    return "c_set";
+  }
+
+  public String getC_queueFN() {
+    return "c_queue";
+  }
+
+  public String getC_dequeFN() {
+    return "c_deque";
+  }
+
+  public String getC_stackFN() {
+    return "c_stack";
+  }
+
+  public String getM_empByCityFN() {
+    return "m_empByCity";
+  }
+
+  public String getDayFN() {
+    return "day";
+  }
+
+  @Override
+  public void fromData(PdxReader in) {
+    this.p_bool = in.readBoolean("p_bool");
+    this.p_byte = in.readByte("p_byte");
+    this.p_short = in.readShort("p_short");
+    this.p_int = in.readInt("p_int");
+    this.p_long = in.readLong("p_long");
+    this.p_float = in.readFloat("p_float");
+    this.p_double = in.readDouble("p_double");
+    this.w_bool = in.readBoolean("w_bool");
+    this.w_byte = in.readByte("w_byte");
+    this.w_short = in.readShort("w_short");
+    this.w_int = in.readInt("w_int");
+    this.w_long = in.readLong("w_long");
+    this.w_float = in.readFloat("w_float");
+    this.w_double = in.readDouble("w_double");
+    this.w_string = in.readString("w_string");
+    this.w_bigInt = (BigInteger) in.readObject("w_bigInt");
+    this.w_bigDec = (BigDecimal) in.readObject("w_bigDec");
+
+    // P_Arrays
+    this.p_boolArray = in.readBooleanArray("p_boolArray");
+    this.p_byteArray = in.readByteArray("p_byteArray");
+    this.p_shortArray = in.readShortArray("p_shortArray");
+    this.p_intArray = in.readIntArray("p_intArray");
+    this.p_longArray = in.readLongArray("p_longArray");
+    this.p_floatArray = in.readFloatArray("p_floatArray");
+    this.p_doubleArray = in.readDoubleArray("p_doubleArray");
+
+    // W_Arrays
+    this.w_boolArray = (Boolean[]) in.readObjectArray("w_boolArray");
+    this.w_byteArray = (Byte[]) in.readObjectArray("w_byteArray");
+    this.w_shortArray = (Short[]) in.readObjectArray("w_shortArray");
+    this.w_intArray = (Integer[]) in.readObjectArray("w_intArray");
+    this.w_longArray = (Long[]) in.readObjectArray("w_longArray");
+    this.w_floatArray = (Float[]) in.readObjectArray("w_floatArray");
+    this.w_doubleArray = (Double[]) in.readObjectArray("w_doubleArray");
+    this.w_strArray = in.readStringArray("w_strArray");
+    this.w_bigIntArray = (BigInteger[]) in.readObjectArray("w_bigIntArray");
+    this.w_bigDecArray = (BigDecimal[]) in.readObjectArray("w_bigDecArray");
+
+    // Collections
+    this.c_list = (List<String>) in.readObject("c_list");
+    this.c_set = (Set<Object>) in.readObject("c_set");
+    this.c_queue = (Queue<String>) in.readObject("c_queue");
+    this.c_deque = (Deque<Integer>) in.readObject("c_deque");
+    this.c_stack = (Stack<String>) in.readObject("c_stack");
+
+    // Map
+    this.m_empByCity = (Map<String, List<Employee>>) in.readObject("m_empByCity");
+
+    // Enum
+    this.day = (Day) (in.readObject("day"));
+    
+    //User Object
+    this.employee = (Employee) in.readObject("employee");
+    //String type= in.readString("@type");
+  }
+
+  @Override
+  public void toData(PdxWriter out) {
+    //if(m_unreadFields != null){ out.writeUnreadFields(m_unreadFields); }
+    out.writeBoolean("p_bool", this.p_bool);
+    out.writeByte("p_byte", this.p_byte);
+    out.writeShort("p_short", p_short);
+    out.writeInt("p_int", p_int);
+    out.writeLong("p_long", p_long);
+    out.writeFloat("p_float", p_float);
+    out.writeDouble("p_double", p_double);
+    out.writeBoolean("w_bool", w_bool);
+    out.writeByte("w_byte", w_byte);
+    out.writeShort("w_short", w_short);
+    out.writeInt("w_int", w_int);
+    out.writeLong("w_long", w_long);
+    out.writeFloat("w_float", w_float);
+    out.writeDouble("w_double", w_double);
+    out.writeString("w_string", w_string);
+    out.writeObject("w_bigInt", w_bigInt);
+    out.writeObject("w_bigDec", w_bigDec);
+
+    // P_Arrays
+    out.writeBooleanArray("p_boolArray", p_boolArray);
+    out.writeByteArray("p_byteArray", p_byteArray);
+    out.writeShortArray("p_shortArray", p_shortArray);
+    out.writeIntArray("p_intArray", p_intArray);
+    out.writeLongArray("p_longArray", p_longArray);
+    out.writeFloatArray("p_floatArray", p_floatArray);
+    out.writeDoubleArray("p_doubleArray", p_doubleArray);
+
+    // W_Arrays
+    out.writeObjectArray("w_boolArray", w_boolArray);
+    out.writeObjectArray("w_byteArray", w_byteArray);
+    out.writeObjectArray("w_shortArray", w_shortArray);
+    out.writeObjectArray("w_intArray", w_intArray);
+    out.writeObjectArray("w_longArray", w_longArray);
+    out.writeObjectArray("w_floatArray", w_floatArray);
+    out.writeObjectArray("w_doubleArray", w_doubleArray);
+    out.writeStringArray("w_strArray", w_strArray);
+    out.writeObjectArray("w_bigIntArray", w_bigIntArray);
+    out.writeObjectArray("w_bigDecArray", w_bigDecArray);
+
+    // Collections
+    out.writeObject("c_list", c_list);
+    out.writeObject("c_set", c_set);
+    out.writeObject("c_queue", c_queue);
+    out.writeObject("c_deque", c_deque);
+    out.writeObject("c_stack", c_stack);
+
+    // Map
+    out.writeObject("m_empByCity", m_empByCity);
+
+    // Enum
+    out.writeObject("day", day);
+
+    out.writeObject("employee", this.employee);
+    //out.writeString("@type", "com.gemstone.gemfire.pdx.TestObjectForJSONFormatter");
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+
+    TestObjectForJSONFormatter other = (TestObjectForJSONFormatter) obj;
+
+    // primitive type
+    if (p_bool != other.p_bool)
+      return false;
+    if (p_byte != other.p_byte)
+      return false;
+    if (p_short != other.p_short)
+      return false;
+    if (p_int != other.p_int)
+      return false;
+    if (p_long != other.p_long)
+      return false;
+    if (p_float != other.p_float)
+      return false;
+    if (p_double != other.p_double)
+      return false;
+
+    // wrapper type
+    if (w_bool.booleanValue() != other.w_bool.booleanValue())
+      return false;
+    if (w_byte.byteValue() != other.w_byte.byteValue())
+      return false;
+    if (w_short.shortValue() != other.w_short.shortValue())
+      return false;
+    if (w_int.intValue() != other.w_int.intValue())
+      return false;
+    if (w_long.longValue() != other.w_long.longValue())
+      return false;
+    if (w_float.floatValue() != other.w_float.floatValue())
+      return false;
+    if (w_double.doubleValue() != other.w_double.doubleValue())
+      return false;
+    if (!w_string.equals(other.w_string))
+      return false;
+    if (w_bigInt.longValue() != other.w_bigInt.longValue())
+      return false;
+    if (w_bigDec.longValue() != other.w_bigDec.longValue())
+      return false;
+
+    // Primitive arrays
+    if (!Arrays.equals(p_boolArray, other.p_boolArray))
+      return false;
+    if (!Arrays.equals(p_byteArray, other.p_byteArray))
+      return false;
+    if (!Arrays.equals(p_shortArray, other.p_shortArray))
+      return false;
+    if (!Arrays.equals(p_intArray, other.p_intArray))
+      return false;
+    if (!Arrays.equals(p_longArray, other.p_longArray))
+      return false;
+    if (!Arrays.equals(p_floatArray, other.p_floatArray))
+      return false;
+    if (!Arrays.equals(p_doubleArray, other.p_doubleArray))
+      return false;
+
+    // wrapper Arrays
+    if (!Arrays.equals(w_boolArray, other.w_boolArray))
+      return false;
+    if (!Arrays.equals(w_byteArray, other.w_byteArray))
+      return false;
+    if (!Arrays.equals(w_shortArray, other.w_shortArray))
+      return false;
+    if (!Arrays.equals(w_intArray, other.w_intArray))
+      return false;
+    if (!Arrays.equals(w_longArray, other.w_longArray))
+      return false;
+    if (!Arrays.equals(w_floatArray, other.w_floatArray))
+      return false;
+    if (!Arrays.equals(w_doubleArray, other.w_doubleArray))
+      return false;
+    if (!Arrays.equals(w_strArray, other.w_strArray))
+      return false;
+    if (!Arrays.equals(w_bigIntArray, other.w_bigIntArray))
+      return false;
+    if (!Arrays.equals(w_bigDecArray, other.w_bigDecArray))
+      return false;
+
+    // comparing Collections based on content, order not considered
+    if (!(c_list.size() == other.c_list.size()
+        && c_list.containsAll(other.c_list) 
+        && other.c_list.containsAll(c_list)))
+      return false;
+    if (!(c_set.size() == other.c_set.size()
+        && c_set.containsAll(other.c_set) 
+        && other.c_set.containsAll(c_set)))
+      return false;
+    if (!(c_queue.size() == other.c_queue.size()
+        && c_queue.containsAll(other.c_queue)
+        && other.c_queue.containsAll(c_queue)))
+      return false;
+    if (!(c_deque.size() == other.c_deque.size()
+        && c_deque.containsAll(other.c_deque) 
+        && other.c_deque.containsAll(c_deque)))
+      return false;
+    
+    // map comparision.
+    if (!(compareMaps(m_empByCity, other.m_empByCity)))
+      return false;
+
+    // Enum validation
+    if (!(day.equals(other.day)))
+      return false;
+    
+    return true;
+  }
+
+  boolean compareMaps(Map m1, Map m2) {
+    if (m1.size() != m2.size())
+      return false;
+    for (Object key : m1.keySet())
+      if (!m1.get(key).equals(m2.get(key)))
+        return false;
+    return true;
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d23a695/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForPdxFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForPdxFormatter.java b/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForPdxFormatter.java
deleted file mode 100644
index 462a572..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/pdx/TestObjectForPdxFormatter.java
+++ /dev/null
@@ -1,1003 +0,0 @@
-/*
- * 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.pdx;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.Set;
-import java.util.Stack;
-
-import com.gemstone.gemfire.pdx.PdxReader;
-import com.gemstone.gemfire.pdx.PdxSerializable;
-import com.gemstone.gemfire.pdx.PdxWriter;
-import org.json.JSONException;
-import org.json.JSONObject;
-
-enum Day {
-  Sunday, Monday, Tuesday, Wednesday, Thursday, Friday, Saturday
-}
-
-public class TestObjectForPdxFormatter implements PdxSerializable {
-
-  private boolean p_bool;
-  private byte p_byte;
-  private short p_short;
-  private int p_int;
-  private long p_long;
-  private float p_float;
-  private double p_double;
-  
-  //wrapper
-  private Boolean w_bool;
-  private Byte w_byte;
-  private Short w_short;
-  private Integer w_int;
-  private Long w_long;
-  private BigInteger w_bigInt;
-  private Float w_float;
-  private BigDecimal w_bigDec;
-  private Double w_double;
-  private String w_string;
-  
-  //Primitive_Arrays
-  private boolean[] p_boolArray;
-  private byte[] p_byteArray;
-  private short[] p_shortArray;
-  private int[] p_intArray;
-  private long[] p_longArray;
-  private float[] p_floatArray;
-  private double[] p_doubleArray;
-   
-  //Wrapper_Arrays 
-  private Boolean[] w_boolArray;
-  private Byte[] w_byteArray;
-  private Short[] w_shortArray;
-  private Integer[] w_intArray;
-  private Long[] w_longArray;
-  private BigInteger[] w_bigIntArray;
-  private Float[] w_floatArray;
-  private BigDecimal[] w_bigDecArray;
-  private Double[] w_doubleArray;
-  private String [] w_strArray;
-  
-  //Collection Type: List, Set, Queue, Deque
-  private List<String> c_list;
-  private Set<Object> c_set;
-  private Queue<String> c_queue;
-  private Deque<Integer> c_deque;
-  private Stack<String> c_stack;
-  
-  //Map - Classify Person objects by city
-  Map<String, List<Employee>> m_empByCity;
-  
-  //Enum
-  private Day day;
-
-  private Employee employee;
-  
-  public TestObjectForPdxFormatter(){
-  }
-  
-  public String addClassTypeToJson(String json) throws JSONException {
-    JSONObject jsonObj = new JSONObject(json);
-    jsonObj.put("@type", "com.gemstone.gemfire.pdx.TestObjectForPdxFormatter");
-    return jsonObj.toString();
-  }
-  
-  public void defaultInitialization(){
-    
-    employee = new Employee(1010L, "NilkanthKumar", "Patel");  
-    
-    //Initialize Map type member
-    Employee e1 = new Employee(1L, "Nilkanth", "Patel");
-    Employee e2 = new Employee(2L, "Amey", "Barve");
-    Employee e3 = new Employee(3L, "Shankar", "Hundekar");
-    Employee e4 = new Employee(4L, "Avinash", "Dongre");
-    Employee e5 = new Employee(5L, "supriya", "Patil");
-    Employee e6 = new Employee(6L, "Rajesh", "Kumar");
-    Employee e7 = new Employee(7L, "Vishal", "Rao");
-    Employee e8 = new Employee(8L, "Hitesh", "Khamesara");
-    Employee e9 = new Employee(9L, "Sudhir", "Menon");
-    
-    m_empByCity = new HashMap<String, List<Employee>>();
-    List<Employee> list1 = new ArrayList<Employee>();
-    List<Employee> list2 = new ArrayList<Employee>();
-    List<Employee> list3 = new ArrayList<Employee>();
-    
-    list1.add(e1);
-    list1.add(e2);
-    list1.add(e3);
-    
-    list2.add(e4);
-    list2.add(e5);
-    list2.add(e6);
-    
-    list3.add(e7);
-    list3.add(e8);
-    list3.add(e9);
-    
-    m_empByCity.put("Ahmedabad", list1);
-    m_empByCity.put("mumbai", list2);
-    m_empByCity.put("Pune", list3);
-    
-    
-    //Initialize Collection types members
-    c_list = new ArrayList<String>();
-    c_list.add("Java");
-    c_list.add("scala");
-    c_list.add("closure");
-    
-    c_set = new HashSet<Object>();
-    c_set.add("element 0");
-    c_set.add("element 1");
-    c_set.add("element 2");
-    
-    c_queue = new PriorityQueue<String>(3);
-    c_queue.add("short");
-    c_queue.add("very long indeed");
-    c_queue.add("medium");
-    
-    c_deque = new ArrayDeque<Integer>(4);
-    c_deque.add(15);
-    c_deque.add(30);
-    c_deque.add(20);
-    c_deque.add(18);
-    
-    c_stack = new Stack();
-    c_stack.push( "bat" );
-    c_stack.push( "cat" );
-    c_stack.push( "dog" );
-    
-    //Initialize primitive types members
-    p_bool = true;
-    p_byte = 101;
-    p_short = 32001;
-    p_int = 100001;
-    p_long = 1234567898765432L;
-    p_float = 123.456f;
-    p_double = 98765.12345d; 
-    
-    //Wrapper type member initialization
-    w_bool = new Boolean(false);
-    w_byte = new Byte((byte)11);
-    w_short = new Short((short)101);
-    w_int = new Integer(1001);
-    w_long = new Long(987654321234567L);
-    w_bigInt = new BigInteger("12345678910");
-    w_float = new Float(789.456f);
-    w_bigDec = new BigDecimal(8866333);
-    w_double = new Double(123456.9876d);
-    w_string = new String("Nilkanth Patel");
-  
-    //Initialization for members of type primitive arrays
-    p_boolArray = new boolean[]{ true, false, false};
-    p_byteArray = new byte[]{10, 11, 12};
-    p_shortArray = new short[]{101, 102, 103};
-    p_intArray = new int[]{1001,1002, 1003, 1004, 1005, 1006};
-    p_longArray = new long[]{ 12345678910L, 12345678911L, 12345678912L };
-    p_floatArray = new float[]{ 123.45f, 456.78f, -91011.123f};
-    p_doubleArray = new double[]{1234.5678d, -91011.1213d, 1415.1617d };
-  
-    //Initialization for members of type wrapper arrays
-    w_boolArray = new Boolean[3];
-    w_byteArray = new Byte[3];
-    w_shortArray = new Short[3];
-    w_intArray = new Integer[3];
-    w_longArray = new Long[3];
-    w_floatArray = new Float[3];
-    w_doubleArray = new Double[3];
-    w_strArray = new String[3];
-  
-    for (int i=0; i< 3; i++){
-      w_boolArray[i] = p_boolArray[i];
-      w_byteArray[i] = p_byteArray[i];
-      w_shortArray[i] = p_shortArray[i];
-      w_intArray[i] =  p_intArray[i];
-      w_longArray[i] = p_longArray[i];
-      w_floatArray[i] =  p_floatArray[i];
-      w_doubleArray[i] = p_doubleArray[i];
-    }
-  
-    w_bigIntArray = new BigInteger[] {BigInteger.ZERO, BigInteger.ONE, new BigInteger("12345678910")};
-    w_bigDecArray = new BigDecimal[] {BigDecimal.TEN, new BigDecimal("143.145"), new BigDecimal("10.01")};
-    w_strArray = new String[]{"Nilkanth", "Vishal", "Hitesh"};
-  
-    //Enum type initialization
-    day = Day.Thursday;
-  }
-
-  public TestObjectForPdxFormatter(boolean p_bool, byte p_byte, short p_short,
-      int p_int, long p_long, float p_float, double p_double, Boolean w_bool,
-      Byte w_byte, Short w_short, Integer w_int, Long w_long,
-      BigInteger w_bigInt, Float w_float, BigDecimal w_bigDec,
-      Double w_double, String w_string) {
-    super();
-    this.p_bool = p_bool;
-    this.p_byte = p_byte;
-    this.p_short = p_short;
-    this.p_int = p_int;
-    this.p_long = p_long;
-    this.p_float = p_float;
-    this.p_double = p_double;
-    this.w_bool = w_bool;
-    this.w_byte = w_byte;
-    this.w_short = w_short;
-    this.w_int = w_int;
-    this.w_long = w_long;
-    this.w_bigInt = w_bigInt;
-    this.w_float = w_float;
-    this.w_bigDec = w_bigDec;
-    this.w_double = w_double;
-    this.w_string = w_string;
-  }
-
-  public Employee getEmployee() {
-    return employee;
-  }
-
-  public void setEmployee(Employee employee) {
-    this.employee = employee;
-  }
-  
-  public List<String> getC_list() {
-    return c_list;
-  }
-
-  public void setC_list(List<String> c_list) {
-    this.c_list = c_list;
-  }
-
-  public Set<Object> getC_set() {
-    return c_set;
-  }
-
-  public void setC_set(Set<Object> c_set) {
-    this.c_set = c_set;
-  }
-
-  public Queue<String> getC_queue() {
-    return c_queue;
-  }
-
-  public void setC_queue(Queue<String> c_queue) {
-    this.c_queue = c_queue;
-  }
-
-  public Deque<Integer> getC_deque() {
-    return c_deque;
-  }
-
-  public void setC_deque(Deque<Integer> c_deque) {
-    this.c_deque = c_deque;
-  }
-
-  public Map<String, List<Employee>> getM_empByCity() {
-    return m_empByCity;
-  }
-
-  public void setM_empByCity(Map<String, List<Employee>> m_empByCity) {
-    this.m_empByCity = m_empByCity;
-  }
-
-  public Day getDay() {
-    return day;
-  }
-
-  public void setDay(Day day) {
-    this.day = day;
-  }
-
-  public boolean isP_bool() {
-    return p_bool;
-  }
-        
-  public void setP_bool(boolean p_bool) {
-    this.p_bool = p_bool;
-  }
-  
-  public byte getP_byte() {
-    return p_byte;
-  }
-  
-  public void setP_byte(byte p_byte) {
-    this.p_byte = p_byte;
-  }
-  
-  public short getP_short() {
-    return p_short;
-  }
-  
-  public void setP_short(short p_short) {
-    this.p_short = p_short;
-  }
-  
-  public int getP_int() {
-    return p_int;
-  }
-  
-  public void setP_int(int p_int) {
-    this.p_int = p_int;
-  }
-  
-  public long getP_long() {
-    return p_long;
-  }
-  
-  public void setP_long(long p_long) {
-    this.p_long = p_long;
-  }
-  
-  public float getP_float() {
-    return p_float;
-  }
-  
-  public void setP_float(float p_float) {
-    this.p_float = p_float;
-  }
-  
-  public double getP_double() {
-    return p_double;
-  }
-  
-  public void setP_double(double p_double) {
-    this.p_double = p_double;
-  }
-  
-  public Boolean getW_bool() {
-    return w_bool;
-  }
-  
-  public void setW_bool(Boolean w_bool) {
-    this.w_bool = w_bool;
-  }
-  
-  public Byte getW_byte() {
-    return w_byte;
-  }
-  
-  public void setW_byte(Byte w_byte) {
-    this.w_byte = w_byte;
-  }
-  
-  public Short getW_short() {
-    return w_short;
-  }
-  
-  public void setW_short(Short w_short) {
-    this.w_short = w_short;
-  }
-  
-  public Integer getW_int() {
-    return w_int;
-  }
-  
-  public void setW_int(Integer w_int) {
-    this.w_int = w_int;
-  }
-  
-  public Long getW_long() {
-    return w_long;
-  }
-  
-  public void setW_long(Long w_long) {
-    this.w_long = w_long;
-  }
-  
-  public BigInteger getW_bigInt() {
-    return w_bigInt;
-  }
-  
-  public void setW_bigInt(BigInteger w_bigInt) {
-    this.w_bigInt = w_bigInt;
-  }
-  
-  public Float getW_float() {
-    return w_float;
-  }
-  
-  public void setW_float(Float w_float) {
-    this.w_float = w_float;
-  }
-  
-  public BigDecimal getW_bigDec() {
-    return w_bigDec;
-  }
-  
-  public void setW_bigDec(BigDecimal w_bigDec) {
-    this.w_bigDec = w_bigDec;
-  }
-  
-  public Double getW_double() {
-    return w_double;
-  }
-  
-  public void setW_double(Double w_double) {
-    this.w_double = w_double;
-  }
-  
-  public String getW_string() {
-    return w_string;
-  }
-  
-  public void setW_string(String w_string) {
-    this.w_string = w_string;
-  }
-
-  public boolean[] getP_boolArray() {
-    return p_boolArray;
-  }
-
-  public void setP_boolArray(boolean[] p_boolArray) {
-    this.p_boolArray = p_boolArray;
-  }
-
-  public byte[] getP_byteArray() {
-    return p_byteArray;
-  }
-
-  public void setP_byteArray(byte[] p_byteArray) {
-    this.p_byteArray = p_byteArray;
-  }
-
-  public short[] getP_shortArray() {
-    return p_shortArray;
-  }
-
-  public void setP_shortArray(short[] p_shortArray) {
-    this.p_shortArray = p_shortArray;
-  }
-
-  public int[] getP_intArray() {
-    return p_intArray;
-  }
-
-  public void setP_intArray(int[] p_intArray) {
-    this.p_intArray = p_intArray;
-  }
-
-  public long[] getP_longArray() {
-    return p_longArray;
-  }
-
-  public void setP_longArray(long[] p_longArray) {
-    this.p_longArray = p_longArray;
-  }
-
-  public float[] getP_floatArray() {
-    return p_floatArray;
-  }
-
-  public void setP_floatArray(float[] p_floatArray) {
-    this.p_floatArray = p_floatArray;
-  }
-
-  public double[] getP_doubleArray() {
-    return p_doubleArray;
-  }
-
-  public void setP_doubleArray(double[] p_doubleArray) {
-    this.p_doubleArray = p_doubleArray;
-  }
-
-  public Boolean[] getW_boolArray() {
-    return w_boolArray;
-  }
-
-  public void setW_boolArray(Boolean[] w_boolArray) {
-    this.w_boolArray = w_boolArray;
-  }
-
-  public Byte[] getW_byteArray() {
-    return w_byteArray;
-  }
-
-  public void setW_byteArray(Byte[] w_byteArray) {
-    this.w_byteArray = w_byteArray;
-  }
-
-  public Short[] getW_shortArray() {
-    return w_shortArray;
-  }
-
-  public void setW_shortArray(Short[] w_shortArray) {
-    this.w_shortArray = w_shortArray;
-  }
-
-  public Integer[] getW_intArray() {
-    return w_intArray;
-  }
-
-  public void setW_intArray(Integer[] w_intArray) {
-    this.w_intArray = w_intArray;
-  }
-
-  public Long[] getW_longArray() {
-    return w_longArray;
-  }
-
-  public void setW_longArray(Long[] w_longArray) {
-    this.w_longArray = w_longArray;
-  }
-
-  public BigInteger[] getW_bigIntArray() {
-    return w_bigIntArray;
-  }
-
-  public void setW_bigIntArray(BigInteger[] w_bigIntArray) {
-    this.w_bigIntArray = w_bigIntArray;
-  }
-
-  public Float[] getW_floatArray() {
-    return w_floatArray;
-  }
-
-  public void setW_floatArray(Float[] w_floatArray) {
-    this.w_floatArray = w_floatArray;
-  }
-
-  public BigDecimal[] getW_bigDecArray() {
-    return w_bigDecArray;
-  }
-
-  public void setW_bigDecArray(BigDecimal[] w_bigDecArray) {
-    this.w_bigDecArray = w_bigDecArray;
-  }
-
-  public Double[] getW_doubleArray() {
-    return w_doubleArray;
-  }
-
-  public void setW_doubleArray(Double[] w_doubleArray) {
-    this.w_doubleArray = w_doubleArray;
-  }
-
-  public String[] getW_strArray() {
-    return w_strArray;
-  }
-
-  public void setW_strArray(String[] w_strArray) {
-    this.w_strArray = w_strArray;
-  }
-
-  public Stack<String> getC_stack() {
-    return c_stack;
-  }
-
-  public void setC_stack(Stack<String> c_stack) {
-    this.c_stack = c_stack;
-  }
-
-  // Getters for returning field names
-  public String getP_boolFN() {
-    return "p_bool";
-  }
-
-  public String getP_byteFN() {
-    return "p_byte";
-  }
-
-  public String getP_shortFN() {
-    return "p_short";
-  }
-
-  public String getP_intFN() {
-    return "p_int";
-  }
-
-  public String getP_longFN() {
-    return "p_long";
-  }
-
-  public String getP_floatFn() {
-    return "p_float";
-  }
-
-  public String getP_doubleFN() {
-    return "p_double";
-  }
-
-  public String getW_boolFN() {
-    return "w_bool";
-  }
-
-  public String getW_byteFN() {
-    return "w_byte";
-  }
-
-  public String getW_shortFN() {
-    return "w_short";
-  }
-
-  public String getW_intFN() {
-    return "w_int";
-  }
-
-  public String getW_longFN() {
-    return "w_long";
-  }
-
-  public String getW_bigIntFN() {
-    return "w_bigInt";
-  }
-
-  public String getW_floatFN() {
-    return "w_float";
-  }
-
-  public String getW_bigDecFN() {
-    return "w_bigDec";
-  }
-
-  public String getW_doubleFN() {
-    return "w_double";
-  }
-
-  public String getW_stringFN() {
-    return "w_string";
-  }
-
-  public String getP_boolArrayFN() {
-    return "p_boolArray";
-  }
-
-  public String getP_byteArrayFN() {
-    return "p_byteArray";
-  }
-
-  public String getP_shortArrayFN() {
-    return "p_shortArray";
-  }
-
-  public String getP_intArrayFN() {
-    return "p_intArray";
-  }
-
-  public String getP_longArrayFN() {
-    return "p_longArray";
-  }
-
-  public String getP_floatArrayFN() {
-    return "p_floatArray";
-  }
-
-  public String getP_doubleArrayFN() {
-    return "p_doubleArray";
-  }
-
-  public String getW_boolArrayFN() {
-    return "w_boolArray";
-  }
-
-  public String getW_byteArrayFN() {
-    return "w_byteArray";
-  }
-
-  public String getW_shortArrayFN() {
-    return "w_shortArray";
-  }
-
-  public String getW_intArrayFN() {
-    return "w_intArray";
-  }
-
-  public String getW_longArrayFN() {
-    return "w_longArray";
-  }
-
-  public String getW_bigIntArrayFN() {
-    return "w_bigIntArray";
-  }
-
-  public String getW_floatArrayFN() {
-    return "w_floatArray";
-  }
-
-  public String getW_bigDecArrayFN() {
-    return "w_bigDecArray";
-  }
-
-  public String getW_doubleArrayFN() {
-    return "w_doubleArray";
-  }
-
-  public String getW_strArrayFN() {
-    return "w_strArray";
-  }
-
-  public String getC_listFN() {
-    return "c_list";
-  }
-
-  public String getC_setFN() {
-    return "c_set";
-  }
-
-  public String getC_queueFN() {
-    return "c_queue";
-  }
-
-  public String getC_dequeFN() {
-    return "c_deque";
-  }
-
-  public String getC_stackFN() {
-    return "c_stack";
-  }
-
-  public String getM_empByCityFN() {
-    return "m_empByCity";
-  }
-
-  public String getDayFN() {
-    return "day";
-  }
-
-  @Override
-  public void fromData(PdxReader in) {
-    this.p_bool = in.readBoolean("p_bool");
-    this.p_byte = in.readByte("p_byte");
-    this.p_short = in.readShort("p_short");
-    this.p_int = in.readInt("p_int");
-    this.p_long = in.readLong("p_long");
-    this.p_float = in.readFloat("p_float");
-    this.p_double = in.readDouble("p_double");
-    this.w_bool = in.readBoolean("w_bool");
-    this.w_byte = in.readByte("w_byte");
-    this.w_short = in.readShort("w_short");
-    this.w_int = in.readInt("w_int");
-    this.w_long = in.readLong("w_long");
-    this.w_float = in.readFloat("w_float");
-    this.w_double = in.readDouble("w_double");
-    this.w_string = in.readString("w_string");
-    this.w_bigInt = (BigInteger) in.readObject("w_bigInt");
-    this.w_bigDec = (BigDecimal) in.readObject("w_bigDec");
-
-    // P_Arrays
-    this.p_boolArray = in.readBooleanArray("p_boolArray");
-    this.p_byteArray = in.readByteArray("p_byteArray");
-    this.p_shortArray = in.readShortArray("p_shortArray");
-    this.p_intArray = in.readIntArray("p_intArray");
-    this.p_longArray = in.readLongArray("p_longArray");
-    this.p_floatArray = in.readFloatArray("p_floatArray");
-    this.p_doubleArray = in.readDoubleArray("p_doubleArray");
-
-    // W_Arrays
-    this.w_boolArray = (Boolean[]) in.readObjectArray("w_boolArray");
-    this.w_byteArray = (Byte[]) in.readObjectArray("w_byteArray");
-    this.w_shortArray = (Short[]) in.readObjectArray("w_shortArray");
-    this.w_intArray = (Integer[]) in.readObjectArray("w_intArray");
-    this.w_longArray = (Long[]) in.readObjectArray("w_longArray");
-    this.w_floatArray = (Float[]) in.readObjectArray("w_floatArray");
-    this.w_doubleArray = (Double[]) in.readObjectArray("w_doubleArray");
-    this.w_strArray = in.readStringArray("w_strArray");
-    this.w_bigIntArray = (BigInteger[]) in.readObjectArray("w_bigIntArray");
-    this.w_bigDecArray = (BigDecimal[]) in.readObjectArray("w_bigDecArray");
-
-    // Collections
-    this.c_list = (List<String>) in.readObject("c_list");
-    this.c_set = (Set<Object>) in.readObject("c_set");
-    this.c_queue = (Queue<String>) in.readObject("c_queue");
-    this.c_deque = (Deque<Integer>) in.readObject("c_deque");
-    this.c_stack = (Stack<String>) in.readObject("c_stack");
-
-    // Map
-    this.m_empByCity = (Map<String, List<Employee>>) in.readObject("m_empByCity");
-
-    // Enum
-    this.day = (Day) (in.readObject("day"));
-    
-    //User Object
-    this.employee = (Employee) in.readObject("employee");
-    //String type= in.readString("@type");
-  }
-
-  @Override
-  public void toData(PdxWriter out) {
-    //if(m_unreadFields != null){ out.writeUnreadFields(m_unreadFields); }
-    out.writeBoolean("p_bool", this.p_bool);
-    out.writeByte("p_byte", this.p_byte);
-    out.writeShort("p_short", p_short);
-    out.writeInt("p_int", p_int);
-    out.writeLong("p_long", p_long);
-    out.writeFloat("p_float", p_float);
-    out.writeDouble("p_double", p_double);
-    out.writeBoolean("w_bool", w_bool);
-    out.writeByte("w_byte", w_byte);
-    out.writeShort("w_short", w_short);
-    out.writeInt("w_int", w_int);
-    out.writeLong("w_long", w_long);
-    out.writeFloat("w_float", w_float);
-    out.writeDouble("w_double", w_double);
-    out.writeString("w_string", w_string);
-    out.writeObject("w_bigInt", w_bigInt);
-    out.writeObject("w_bigDec", w_bigDec);
-
-    // P_Arrays
-    out.writeBooleanArray("p_boolArray", p_boolArray);
-    out.writeByteArray("p_byteArray", p_byteArray);
-    out.writeShortArray("p_shortArray", p_shortArray);
-    out.writeIntArray("p_intArray", p_intArray);
-    out.writeLongArray("p_longArray", p_longArray);
-    out.writeFloatArray("p_floatArray", p_floatArray);
-    out.writeDoubleArray("p_doubleArray", p_doubleArray);
-
-    // W_Arrays
-    out.writeObjectArray("w_boolArray", w_boolArray);
-    out.writeObjectArray("w_byteArray", w_byteArray);
-    out.writeObjectArray("w_shortArray", w_shortArray);
-    out.writeObjectArray("w_intArray", w_intArray);
-    out.writeObjectArray("w_longArray", w_longArray);
-    out.writeObjectArray("w_floatArray", w_floatArray);
-    out.writeObjectArray("w_doubleArray", w_doubleArray);
-    out.writeStringArray("w_strArray", w_strArray);
-    out.writeObjectArray("w_bigIntArray", w_bigIntArray);
-    out.writeObjectArray("w_bigDecArray", w_bigDecArray);
-
-    // Collections
-    out.writeObject("c_list", c_list);
-    out.writeObject("c_set", c_set);
-    out.writeObject("c_queue", c_queue);
-    out.writeObject("c_deque", c_deque);
-    out.writeObject("c_stack", c_stack);
-
-    // Map
-    out.writeObject("m_empByCity", m_empByCity);
-
-    // Enum
-    out.writeObject("day", day);
-
-    out.writeObject("employee", this.employee);
-    //out.writeString("@type", "com.gemstone.gemfire.pdx.TestObjectForPdxFormatter");
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-
-    TestObjectForPdxFormatter other = (TestObjectForPdxFormatter) obj;
-
-    // primitive type
-    if (p_bool != other.p_bool)
-      return false;
-    if (p_byte != other.p_byte)
-      return false;
-    if (p_short != other.p_short)
-      return false;
-    if (p_int != other.p_int)
-      return false;
-    if (p_long != other.p_long)
-      return false;
-    if (p_float != other.p_float)
-      return false;
-    if (p_double != other.p_double)
-      return false;
-
-    // wrapper type
-    if (w_bool.booleanValue() != other.w_bool.booleanValue())
-      return false;
-    if (w_byte.byteValue() != other.w_byte.byteValue())
-      return false;
-    if (w_short.shortValue() != other.w_short.shortValue())
-      return false;
-    if (w_int.intValue() != other.w_int.intValue())
-      return false;
-    if (w_long.longValue() != other.w_long.longValue())
-      return false;
-    if (w_float.floatValue() != other.w_float.floatValue())
-      return false;
-    if (w_double.doubleValue() != other.w_double.doubleValue())
-      return false;
-    if (!w_string.equals(other.w_string))
-      return false;
-    if (w_bigInt.longValue() != other.w_bigInt.longValue())
-      return false;
-    if (w_bigDec.longValue() != other.w_bigDec.longValue())
-      return false;
-
-    // Primitive arrays
-    if (!Arrays.equals(p_boolArray, other.p_boolArray))
-      return false;
-    if (!Arrays.equals(p_byteArray, other.p_byteArray))
-      return false;
-    if (!Arrays.equals(p_shortArray, other.p_shortArray))
-      return false;
-    if (!Arrays.equals(p_intArray, other.p_intArray))
-      return false;
-    if (!Arrays.equals(p_longArray, other.p_longArray))
-      return false;
-    if (!Arrays.equals(p_floatArray, other.p_floatArray))
-      return false;
-    if (!Arrays.equals(p_doubleArray, other.p_doubleArray))
-      return false;
-
-    // wrapper Arrays
-    if (!Arrays.equals(w_boolArray, other.w_boolArray))
-      return false;
-    if (!Arrays.equals(w_byteArray, other.w_byteArray))
-      return false;
-    if (!Arrays.equals(w_shortArray, other.w_shortArray))
-      return false;
-    if (!Arrays.equals(w_intArray, other.w_intArray))
-      return false;
-    if (!Arrays.equals(w_longArray, other.w_longArray))
-      return false;
-    if (!Arrays.equals(w_floatArray, other.w_floatArray))
-      return false;
-    if (!Arrays.equals(w_doubleArray, other.w_doubleArray))
-      return false;
-    if (!Arrays.equals(w_strArray, other.w_strArray))
-      return false;
-    if (!Arrays.equals(w_bigIntArray, other.w_bigIntArray))
-      return false;
-    if (!Arrays.equals(w_bigDecArray, other.w_bigDecArray))
-      return false;
-
-    // comparing Collections based on content, order not considered
-    if (!(c_list.size() == other.c_list.size()
-        && c_list.containsAll(other.c_list) 
-        && other.c_list.containsAll(c_list)))
-      return false;
-    if (!(c_set.size() == other.c_set.size()
-        && c_set.containsAll(other.c_set) 
-        && other.c_set.containsAll(c_set)))
-      return false;
-    if (!(c_queue.size() == other.c_queue.size()
-        && c_queue.containsAll(other.c_queue)
-        && other.c_queue.containsAll(c_queue)))
-      return false;
-    if (!(c_deque.size() == other.c_deque.size()
-        && c_deque.containsAll(other.c_deque) 
-        && other.c_deque.containsAll(c_deque)))
-      return false;
-    
-    // map comparision.
-    if (!(compareMaps(m_empByCity, other.m_empByCity)))
-      return false;
-
-    // Enum validation
-    if (!(day.equals(other.day)))
-      return false;
-    
-    return true;
-  }
-
-  boolean compareMaps(Map m1, Map m2) {
-    if (m1.size() != m2.size())
-      return false;
-    for (Object key : m1.keySet())
-      if (!m1.get(key).equals(m2.get(key)))
-        return false;
-    return true;
-  }
-}
-
-


[44/50] [abbrv] incubator-geode git commit: GEODE-1397: Rename HelpCommandsJUnitTest to ...IntegrationTest

Posted by kl...@apache.org.
GEODE-1397: Rename HelpCommandsJUnitTest to ...IntegrationTest


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

Branch: refs/heads/feature/GEODE-835
Commit: dd8ea42aa4fa00eda4f8812d8fdb108ceec3687a
Parents: 3a1c1a9
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 18 14:24:39 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu May 19 19:34:51 2016 -0700

----------------------------------------------------------------------
 .../commands/HelpCommandsIntegrationTest.java   | 130 +++++++++++++++++++
 .../cli/commands/HelpCommandsJUnitTest.java     | 130 -------------------
 2 files changed, 130 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dd8ea42a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
new file mode 100644
index 0000000..5d43a0e
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsIntegrationTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.management.internal.cli.commands;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.management.internal.cli.CommandManager;
+import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
+import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Map;
+import java.util.Properties;
+
+import static com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase.commandResultToString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+@Category(IntegrationTest.class)
+public class HelpCommandsIntegrationTest extends JUnit4DistributedTestCase {
+
+  private int jmxPort;
+
+  private Gfsh gfsh;
+
+  @Before
+  public void setup() throws Exception {
+    jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
+
+    Properties localProps = new Properties();
+    localProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
+    localProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
+    localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
+    getSystem(localProps);
+
+    gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
+  }
+
+  @After
+  public void teardown() {
+    disconnectAllFromDS();
+
+    gfsh.executeCommand("disconnect");
+  }
+
+  @Ignore("Disconnect command doesn't appear to be working")
+  @Test
+  public void testOfflineHelp() throws Exception {
+    Properties helpProps = new Properties();
+    helpProps.load(HelpCommandsIntegrationTest.class.getResourceAsStream("golden-help-offline.properties"));
+
+    CommandManager cm = CommandManager.getInstance();
+    for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
+      // Mock commands may have been produced in the VM by other tests
+      // 'quit' is an alias for 'exit' and doesn't produce help
+      if (e.getKey().contains("mock") || e.getKey().contains("quit")) {
+        continue;
+      }
+
+      CommandResult cr = (CommandResult) gfsh.executeCommand("help " + e.getKey()).getResult();
+      String gfshResult = commandResultToString(cr);
+
+      String goldParam = e.getKey().replace(" ", "-") + ".help";
+      String goldResult = helpProps.getProperty(goldParam);
+      assertNotNull("No golden text for: " + goldParam, goldResult);
+      assertEquals(goldResult.trim(), gfshResult.trim());
+
+      helpProps.remove(goldParam);
+    }
+
+    // No help should remain unchecked
+    assertEquals(0, helpProps.size());
+  }
+
+  @Test
+  public void testOnlineHelp() throws Exception {
+    Properties helpProps = new Properties();
+    helpProps.load(HelpCommandsIntegrationTest.class.getResourceAsStream("golden-help-online.properties"));
+
+    gfsh.executeCommand("connect --jmx-manager=localhost[" + jmxPort + "]");
+
+    CommandManager cm = CommandManager.getInstance();
+    for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
+      // Mock commands may have been produced in the VM by other tests
+      // 'quit' is an alias for 'exit' and doesn't produce help
+      if (e.getKey().contains("mock") || e.getKey().contains("quit")) {
+        continue;
+      }
+
+      CommandResult cr = (CommandResult) gfsh.executeCommand("help " + e.getKey()).getResult();
+      String gfshResult = commandResultToString(cr);
+
+      String goldParam = e.getKey().replace(" ", "-") + ".help";
+      String goldResult = helpProps.getProperty(goldParam);
+      assertNotNull("No golden text for: " + goldParam, goldResult);
+
+      String[] lines = gfshResult.split("\n");
+      gfshResult = String.join("\n", lines[0], lines[1], lines[2], lines[3]);
+
+      assertEquals(goldResult.trim(), gfshResult.trim());
+
+      helpProps.remove(goldParam);
+    }
+
+    // No help should remain unchecked
+    assertEquals(0, helpProps.size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dd8ea42a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
deleted file mode 100644
index 1d81d2e..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HelpCommandsJUnitTest.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.management.internal.cli.commands;
-
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.management.internal.cli.CommandManager;
-import com.gemstone.gemfire.management.internal.cli.parser.CommandTarget;
-import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
-import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
-import com.gemstone.gemfire.management.internal.cli.shell.GfshConfig;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.Map;
-import java.util.Properties;
-
-import static com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase.commandResultToString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-@Category(IntegrationTest.class)
-public class HelpCommandsJUnitTest extends JUnit4DistributedTestCase {
-
-  private int jmxPort;
-
-  private Gfsh gfsh;
-
-  @Before
-  public void setup() throws Exception {
-    jmxPort = AvailablePortHelper.getRandomAvailableTCPPort();
-
-    Properties localProps = new Properties();
-    localProps.setProperty(DistributionConfig.JMX_MANAGER_NAME, "true");
-    localProps.setProperty(DistributionConfig.JMX_MANAGER_START_NAME, "true");
-    localProps.setProperty(DistributionConfig.JMX_MANAGER_PORT_NAME, String.valueOf(jmxPort));
-    getSystem(localProps);
-
-    gfsh = Gfsh.getInstance(false, new String[0], new GfshConfig());
-  }
-
-  @After
-  public void teardown() {
-    disconnectAllFromDS();
-
-    gfsh.executeCommand("disconnect");
-  }
-
-  @Ignore("Disconnect command doesn't appear to be working")
-  @Test
-  public void testOfflineHelp() throws Exception {
-    Properties helpProps = new Properties();
-    helpProps.load(HelpCommandsJUnitTest.class.getResourceAsStream("golden-help-offline.properties"));
-
-    CommandManager cm = CommandManager.getInstance();
-    for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
-      // Mock commands may have been produced in the VM by other tests
-      // 'quit' is an alias for 'exit' and doesn't produce help
-      if (e.getKey().contains("mock") || e.getKey().contains("quit")) {
-        continue;
-      }
-
-      CommandResult cr = (CommandResult) gfsh.executeCommand("help " + e.getKey()).getResult();
-      String gfshResult = commandResultToString(cr);
-
-      String goldParam = e.getKey().replace(" ", "-") + ".help";
-      String goldResult = helpProps.getProperty(goldParam);
-      assertNotNull("No golden text for: " + goldParam, goldResult);
-      assertEquals(goldResult.trim(), gfshResult.trim());
-
-      helpProps.remove(goldParam);
-    }
-
-    // No help should remain unchecked
-    assertEquals(0, helpProps.size());
-  }
-
-  @Test
-  public void testOnlineHelp() throws Exception {
-    Properties helpProps = new Properties();
-    helpProps.load(HelpCommandsJUnitTest.class.getResourceAsStream("golden-help-online.properties"));
-
-    gfsh.executeCommand("connect --jmx-manager=localhost[" + jmxPort + "]");
-
-    CommandManager cm = CommandManager.getInstance();
-    for (Map.Entry<String, CommandTarget> e : cm.getCommands().entrySet()) {
-      // Mock commands may have been produced in the VM by other tests
-      // 'quit' is an alias for 'exit' and doesn't produce help
-      if (e.getKey().contains("mock") || e.getKey().contains("quit")) {
-        continue;
-      }
-
-      CommandResult cr = (CommandResult) gfsh.executeCommand("help " + e.getKey()).getResult();
-      String gfshResult = commandResultToString(cr);
-
-      String goldParam = e.getKey().replace(" ", "-") + ".help";
-      String goldResult = helpProps.getProperty(goldParam);
-      assertNotNull("No golden text for: " + goldParam, goldResult);
-
-      String[] lines = gfshResult.split("\n");
-      gfshResult = String.join("\n", lines[0], lines[1], lines[2], lines[3]);
-
-      assertEquals(goldResult.trim(), gfshResult.trim());
-
-      helpProps.remove(goldParam);
-    }
-
-    // No help should remain unchecked
-    assertEquals(0, helpProps.size());
-  }
-}


[36/50] [abbrv] incubator-geode git commit: GEODE-1243: Updates to use new CategoryWithParameterizedRunnerFactory

Posted by kl...@apache.org.
GEODE-1243: Updates to use new CategoryWithParameterizedRunnerFactory


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

Branch: refs/heads/feature/GEODE-835
Commit: aeb8c01df213b8f43a10c6e9bdf1529c5f723ffc
Parents: 6f4a879
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 11 07:42:23 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed May 18 14:39:17 2016 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java | 3 ++-
 .../distributed/LocatorLauncherLocalFileIntegrationTest.java    | 2 ++
 .../distributed/LocatorLauncherLocalIntegrationTest.java        | 2 ++
 .../distributed/LocatorLauncherRemoteFileIntegrationTest.java   | 2 ++
 .../distributed/LocatorLauncherRemoteIntegrationTest.java       | 2 ++
 .../LocatorLauncherRemoteWithCustomLoggingIntegrationTest.java  | 5 +++++
 6 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/aeb8c01d/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
index 1d7afae..d25427a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorJUnitTest.java
@@ -31,8 +31,8 @@ import java.util.List;
 import java.util.Properties;
 import java.util.function.IntSupplier;
 
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -60,6 +60,7 @@ import org.junit.runners.Parameterized;
 
 @Category(IntegrationTest.class)
 @RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorJUnitTest {
 
   private static final int REQUEST_TIMEOUT = 5 * 1000;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/aeb8c01d/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
index 422fb7c..6451a67 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalFileIntegrationTest.java
@@ -18,6 +18,7 @@ package com.gemstone.gemfire.distributed;
 
 import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -36,6 +37,7 @@ import static org.junit.Assert.assertFalse;
  */
 @Category(IntegrationTest.class)
 @RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherLocalFileIntegrationTest extends LocatorLauncherLocalIntegrationTest {
 
   @Before

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/aeb8c01d/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
index 9137207..a8998f1 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherLocalIntegrationTest.java
@@ -23,6 +23,7 @@ import java.lang.management.ManagementFactory;
 import java.net.BindException;
 import java.net.InetAddress;
 
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -53,6 +54,7 @@ import org.junit.runners.Parameterized;
  */
 @Category(IntegrationTest.class)
 @RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherLocalIntegrationTest extends AbstractLocatorLauncherIntegrationTestCase {
 
   @Before

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/aeb8c01d/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
index 13e936d..4d8b94b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileIntegrationTest.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -47,6 +48,7 @@ import org.junit.runners.Parameterized;
  */
 @Category(IntegrationTest.class)
 @RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherRemoteFileIntegrationTest extends LocatorLauncherRemoteIntegrationTest {
 
   @Before

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/aeb8c01d/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
index 508c13c..5cd1fcf 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteIntegrationTest.java
@@ -29,6 +29,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -60,6 +61,7 @@ import org.junit.runners.Parameterized;
  */
 @Category(IntegrationTest.class)
 @RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherRemoteIntegrationTest extends AbstractLocatorLauncherRemoteIntegrationTestCase {
 
   protected volatile Process process;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/aeb8c01d/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteWithCustomLoggingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteWithCustomLoggingIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteWithCustomLoggingIntegrationTest.java
index 2e84905..5f024c8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteWithCustomLoggingIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteWithCustomLoggingIntegrationTest.java
@@ -24,6 +24,7 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.gemstone.gemfire.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.junit.Before;
 import org.junit.Rule;
@@ -36,11 +37,15 @@ import com.gemstone.gemfire.internal.process.ProcessStreamReader;
 import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 /**
  * Integration tests for launching a Locator in a forked process with custom logging configuration
  */
 @Category(IntegrationTest.class)
+@RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
 public class LocatorLauncherRemoteWithCustomLoggingIntegrationTest extends AbstractLocatorLauncherRemoteIntegrationTestCase {
 
   private File customConfigFile;


[04/50] [abbrv] incubator-geode git commit: GEODE-1293: Updating module scripts and structure

Posted by kl...@apache.org.
GEODE-1293: Updating module scripts and structure

- Remove cacheserver.sh and gemfire.sh scripts in favor of using gfsh
- Adjust p2p config to always use locators
- Have modify_war reference correct jars


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

Branch: refs/heads/feature/GEODE-835
Commit: d216e55330e794137cee2684d05eaef683ff00f8
Parents: d1a0748
Author: Jens Deppe <jd...@pivotal.io>
Authored: Tue Mar 29 12:46:22 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri May 13 11:16:16 2016 -0700

----------------------------------------------------------------------
 extensions/geode-modules-assembly/build.gradle  |  53 ++------
 .../release/conf/cache-client.xml               |  14 +-
 .../release/conf/cache-server.xml               |   6 +-
 .../release/scripts/cacheserver.bat             | 133 -------------------
 .../release/scripts/cacheserver.sh              |  97 --------------
 .../release/scripts/gemfire.bat                 |  41 ------
 .../release/scripts/gemfire.sh                  |  58 --------
 .../release/scripts/setenv.properties           |   1 +
 .../release/session/bin/cacheserver.bat         |  56 --------
 .../release/session/bin/cacheserver.sh          |  57 --------
 .../release/session/bin/gemfire.bat             |  53 --------
 .../release/session/bin/gemfire.sh              |  69 ----------
 .../release/session/bin/modify_war              |  91 +++++++------
 .../release/session/bin/setenv.properties       |   6 -
 .../gemfire-cs-tomcat-7/context-fragment.xml    |  15 ---
 .../tcserver/gemfire-cs-tomcat-7/modules.env    |   1 -
 .../gemfire-cs-tomcat-8/context-fragment.xml    |  15 ---
 .../tcserver/gemfire-cs-tomcat-8/modules.env    |   1 -
 .../gemfire-cs/configuration-prompts.properties |  17 ---
 .../tcserver/gemfire-cs/context-fragment.xml    |  13 --
 .../release/tcserver/gemfire-cs/modules.env     |   1 -
 .../tcserver/gemfire-cs/server-fragment.xml     |  12 --
 .../gemfire-p2p-tomcat-7/context-fragment.xml   |  15 ---
 .../tcserver/gemfire-p2p-tomcat-7/modules.env   |   1 -
 .../gemfire-p2p-tomcat-8/context-fragment.xml   |  15 ---
 .../tcserver/gemfire-p2p-tomcat-8/modules.env   |   1 -
 .../configuration-prompts.properties            |  19 ---
 .../tcserver/gemfire-p2p/context-fragment.xml   |  13 --
 .../release/tcserver/gemfire-p2p/modules.env    |   1 -
 .../tcserver/gemfire-p2p/server-fragment.xml    |  14 --
 .../geode-cs-tomcat-7/context-fragment.xml      |  15 +++
 .../geode-cs-tomcat-8/context-fragment.xml      |  15 +++
 .../geode-cs/configuration-prompts.properties   |  17 +++
 .../tcserver/geode-cs/context-fragment.xml      |  13 ++
 .../tcserver/geode-cs/server-fragment.xml       |  12 ++
 .../geode-p2p-tomcat-7/context-fragment.xml     |  15 +++
 .../geode-p2p-tomcat-8/context-fragment.xml     |  15 +++
 .../geode-p2p/configuration-prompts.properties  |  19 +++
 .../tcserver/geode-p2p/context-fragment.xml     |  13 ++
 .../tcserver/geode-p2p/server-fragment.xml      |  14 ++
 extensions/geode-modules-session/build.gradle   |   3 +-
 .../gemstone/gemfire/modules/util/Banner.java   |  20 +--
 geode-core/build.gradle                         |   2 +-
 gradle/dependency-versions.properties           |   1 +
 44 files changed, 225 insertions(+), 838 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/build.gradle b/extensions/geode-modules-assembly/build.gradle
index 960d924..8109466 100644
--- a/extensions/geode-modules-assembly/build.gradle
+++ b/extensions/geode-modules-assembly/build.gradle
@@ -19,15 +19,12 @@ import org.apache.tools.ant.filters.ReplaceTokens
 
 configurations {
   slf4jDeps
-  servletApiDeps
   moduleDistOutputs
 }
 
 dependencies {
   slf4jDeps 'org.slf4j:slf4j-api:' + project.'slf4j-api.version'
   slf4jDeps 'org.slf4j:slf4j-jdk14:' + project.'slf4j-api.version'
-
-  servletApiDeps 'javax.servlet:servlet-api:2.5'
 }
 
 jar.enabled = false
@@ -42,7 +39,7 @@ disableSigning()
 
 def getJarArtifact(module) {
   project(module).configurations.archives.artifacts.findAll {
-    it instanceof PublishArtifact && it.classifier == '' && it.type == 'jar'
+    it instanceof PublishArtifact && it.type == 'jar' && it.classifier == ''
   }.collect { it.file }
 }
 
@@ -59,10 +56,6 @@ def configureTcServerAssembly = {
     from('release/scripts') {
       include '*'
     }
-    from('release/tcserver/geode-cs') {
-      include 'modules.env'
-    }
-    filter(ReplaceTokens, tokens:['GEMFIRE_MODULES_VERSION': version])
   }
   into('geode-cs/conf') {
     from('release/conf') {
@@ -76,8 +69,6 @@ def configureTcServerAssembly = {
   into('geode-cs') {
     from('release/tcserver/geode-cs') {
       include 'configuration-prompts.properties'
-      include 'README.txt'
-      filter(ReplaceTokens, tokens:['VERSION': version])
     }
   }
 
@@ -87,11 +78,6 @@ def configureTcServerAssembly = {
       include 'context-fragment.xml'
     }
   }
-  into('geode-cs-tomcat-7/bin') {
-    from('release/tcserver/geode-cs-tomcat-7') {
-      include 'modules.env'
-    }
-  }
   // All peer-to-peer files
   into('geode-p2p/lib') {
     from getJarArtifact(':extensions/geode-modules')
@@ -106,9 +92,6 @@ def configureTcServerAssembly = {
       include 'gemfire.*'
       include 'setenv.properties'
     }
-    from('release/tcserver/geode-p2p') {
-      include 'modules.env'
-    }
   }
   into('geode-p2p/conf') {
     from('release/conf') {
@@ -122,8 +105,6 @@ def configureTcServerAssembly = {
   into('geode-p2p') {
     from('release/tcserver/geode-p2p') {
       include 'configuration-prompts.properties'
-      include 'README.txt'
-      filter(ReplaceTokens, tokens:['VERSION': version])
     }
   }
 
@@ -133,32 +114,17 @@ def configureTcServerAssembly = {
       include 'context-fragment.xml'
     }
   }
-  into('geode-p2p-tomcat-7/bin') {
-    from('release/tcserver/geode-p2p-tomcat-7') {
-      include 'modules.env'
-    }
-  }
 }
 
 def configureTcServer30Assembly = {
   archiveName = "Apache_Geode_Modules-${version}-tcServer30.zip"
 
-  into('geode-cs-tomcat-8/bin') {
-    from('release/tcserver/geode-cs-tomcat-8') {
-      include 'modules.env'
-    }
-  }
   into('geode-cs-tomcat-8/conf') {
     from('release/tcserver/geode-cs-tomcat-8') {
       include 'context-fragment.xml'
     }
   }
 
-  into('geode-p2p-tomcat-8/bin') {
-    from('release/tcserver/geode-p2p-tomcat-8') {
-      include 'modules.env'
-    }
-  }
   into('geode-p2p-tomcat-8/conf') {
     from('release/tcserver/geode-p2p-tomcat-8') {
       include 'context-fragment.xml'
@@ -201,23 +167,24 @@ task distAppServer(type: Zip, dependsOn: ':extensions/geode-modules-session:asse
   archiveName = "Apache_Geode_Modules-${version}-AppServer.zip"
 
   into('lib') {
+    // Use a closure here so that the evaluation is deferred. Without this the 'internal'
+    // jar won't be picked up.
+    from { getJarArtifact(':extensions/geode-modules-session') }
     from getJarArtifact(':extensions/geode-modules')
-    from getJarArtifact(':extensions/geode-modules-session')
     from configurations.slf4jDeps
-    from configurations.servletApiDeps
   }
 
   into('bin') {
     from('release/session/bin/') {
-      include 'setenv.properties'
       include 'modify_war'
-      include 'cacheserver.*'
-      include 'gemfire.*'
 
-      filter(ReplaceTokens, tokens:['GEMFIRE_MODULES_VERSION': version])
+      filter(ReplaceTokens, tokens:['GEODE_VERSION': version])
       filter(ReplaceTokens, tokens:['SLF4J_VERSION': project.'slf4j-api.version'])
-      filter(ReplaceTokens, tokens:['SERVLET_API_VERSION': project.'javax.servlet-api.version'])
-      filter(ReplaceTokens, tokens:['GEMFIRE_VERSION': version])
+      filter(ReplaceTokens, tokens:['LOG4J_VERSION': project.'log4j.version'])
+      filter(ReplaceTokens, tokens:['FASTUTIL_VERSION': project.'fastutil.version'])
+      filter(ReplaceTokens, tokens:['ANTLR_VERSION': project.'antlr.version'])
+      filter(ReplaceTokens, tokens:['TX_VERSION': project.'javax.transaction-api.version'])
+      filter(ReplaceTokens, tokens:['JGROUPS_VERSION': project.'jgroups.version'])
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/conf/cache-client.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/conf/cache-client.xml b/extensions/geode-modules-assembly/release/conf/cache-client.xml
index c36e0ef..297b6a6 100755
--- a/extensions/geode-modules-assembly/release/conf/cache-client.xml
+++ b/extensions/geode-modules-assembly/release/conf/cache-client.xml
@@ -21,19 +21,19 @@
     xsi:schemaLocation="http://geode.apache.org/schema/cache http://geode.apache.org/schema/cache/cache-1.0.xsd"
     version="1.0">
 
-  <!-- The default pool connects to a cache server running on  localhost at
-       port 40404. To connect to a different server host and port, modify
-       the following pool server host and port. -->
+  <!-- The default pool connects to a locator running on localhost at
+       port 10334. To connect to a different locator host and port, modify
+       the following pool locator host and port. -->
   <pool name="sessions" subscription-enabled="true">
-    <server host="localhost" port="40404"/>
+    <locator host="localhost" port="10334"/>
   </pool>
 
-  <!-- To configure the client to use a locator instead of a server, replace
-       the server pool above with the locator pool below and modify the locator
+  <!-- To configure the client to use a server instead of a locator, replace
+       the locator pool above with the server pool below and modify the server
        host and port as necessary. -->
   <!--
   <pool name="sessions" subscription-enabled="true">
-    <locator host="localhost" port="10334"/>
+    <server host="localhost" port="40404"/>
   </pool>
   -->
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/conf/cache-server.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/conf/cache-server.xml b/extensions/geode-modules-assembly/release/conf/cache-server.xml
index 9dad75f..67ccbd1 100755
--- a/extensions/geode-modules-assembly/release/conf/cache-server.xml
+++ b/extensions/geode-modules-assembly/release/conf/cache-server.xml
@@ -31,10 +31,8 @@
   </gateway-hub>
   -->
 
-  <!-- Uncomment the following cache-server element to modify the listen port -->
-  <!--
-  <cache-server port="44444"/>
-  -->
+  <!-- Change the following cache-server element to modify the listening port -->
+  <cache-server port="0"/>
 
   <!-- Uncomment the following disk-store element to modify the default disk store directory -->
   <!--

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/scripts/cacheserver.bat
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/scripts/cacheserver.bat b/extensions/geode-modules-assembly/release/scripts/cacheserver.bat
deleted file mode 100755
index ef7f99f..0000000
--- a/extensions/geode-modules-assembly/release/scripts/cacheserver.bat
+++ /dev/null
@@ -1,133 +0,0 @@
-@echo off
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem      http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-setlocal
-setlocal enableextensions
-setlocal enabledelayedexpansion
-set scriptdir=%~dp0
-set gf=%scriptdir:\bin\=%
-if exist "%gf%\lib\gemfire.jar" @goto gfok
-echo Could not determine GEMFIRE location
-rem verify other 2>nul
-goto done
-:gfok
-
-if exist "%gf%\bin\modules.env" (
-  FOR /F "tokens=*" %%i in ('type %gf%\bin\modules.env') do SET %%i
-)
-
-rem Check for the -d argument
-set /a FOUND_ARG=0
-set TC_INSTALL_DIR=
-FOR %%A IN (%*) DO (
-  if !FOUND_ARG! == 1 (
-    set /a FOUND_ARG-=1
-    set TC_INSTALL_DIR=%%~fA
-  )
-  if %%A == -d (
-    set /a FOUND_ARG+=1
-  )
-)
-
-rem Pull out the unused args for the java class
-set CLASS_ARGS=
-:loop
-IF "%1"=="" GOTO ENDLOOP
-  if "%1" == "-d" (
-    shift
-    shift
-  ) else ( 
-    set "CLASS_ARGS=!CLASS_ARGS! %1" 
-    shift
-  )
-GOTO loop
-
-:ENDLOOP
-
-IF NOT "%TC_INSTALL_DIR%" == "" goto SET_TOMCAT_DIR
-FOR /f %%f in ('forfiles /P %gf%\.. /m tomcat-%TOMCAT_MAJOR_VER%* /c "cmd /c echo @path"') do set TOMCAT_DIR=%%f
-REM Strip the surrounding quotes
-set TOMCAT_DIR=%TOMCAT_DIR:"=%
-goto TEST_TOMCAT_DIR
-
-:SET_TOMCAT_DIR
-set /p TOMCAT_VER=<"%gf%\conf\tomcat.version"
-set TOMCAT_DIR="!TC_INSTALL_DIR!\tomcat-!TOMCAT_VER!"
-
-:TEST_TOMCAT_DIR
-if not exist "!TOMCAT_DIR!\lib\catalina.jar" goto TOMCAT_NOT_FOUND
-goto FIND_MOD_JAR
-
-:FIND_MOD_JAR
-FOR %%f in (!gf!\lib\geode-modules-?.*.jar) do set MOD_JAR=%%f
-IF NOT "%MOD_JAR%" == "" goto FIND_LOG_API
-rem This is the default modules jar
-set MOD_JAR="!gf!\lib\geode-modules.jar"
-
-:FIND_LOG_API
-FOR %%f in (!gf!\lib\log4j-api*.jar) do set LOG_API_JAR=%%f
-IF NOT "%LOG_API_JAR%" == "" goto FIND_LOG_CORE
-echo ERROR: Log4J API jar not found.
-goto LIBS_NOT_FOUND
-
-:FIND_LOG_CORE
-FOR %%f in (!gf!\lib\log4j-core*.jar) do set LOG_CORE_JAR=%%f
-IF NOT "%LOG_CORE_JAR%" == "" goto MAIN_PROCESSING
-echo ERROR: Log4J Core jar not found.
-goto LIBS_NOT_FOUND
-
-
-:LIBS_NOT_FOUND
-echo ERROR: The required libraries could not be located. 
-echo Try using the -d ^<tc Server installation directory^> option or make sure it was installed correctly.
-echo Example: cacheserver.bat start -d "c:\Program Files\Pivotal\tcServer\pivotal-tc-server-standard"
-exit /b 1
-
-:TOMCAT_NOT_FOUND
-echo ERROR: The TOMCAT libraries could not be located. 
-echo Try using the -d ^<tc Server installation directory^> option or make sure it was installed correctly.
-echo Example: cacheserver.bat start -d "c:\Program Files\Pivotal\tcServer\pivotal-tc-server-standard"
-exit /b 1
-
-:MAIN_PROCESSING
-REM Initialize classpath
-
-REM Add GemFire classes
-set GEMFIRE_JARS=%MOD_JAR%;%LOG_API_JAR%;%LOG_CORE_JAR%;%gf%/lib/gemfire.jar;%gf%/lib/antlr.jar;%gf%/lib/mail.jar
-
-REM Add Tomcat classes
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%TOMCAT_DIR%/lib/servlet-api.jar;%TOMCAT_DIR%/lib/catalina.jar;%gf%/lib/geode-modules.jar;%TOMCAT_DIR%/bin/tomcat-juli.jar;%TOMCAT_DIR%/lib/tomcat-util.jar
-
-REM Add conf directory
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%/conf
-
-
-if defined CLASSPATH set GEMFIRE_JARS=%GEMFIRE_JARS%;%CLASSPATH%
-
-if not defined GF_JAVA (
-  REM %GF_JAVA% is not defined, assume it is on the PATH
-  set GF_JAVA=java
-)
-
-"%GF_JAVA%" %JAVA_ARGS% -classpath "%GEMFIRE_JARS%" com.gemstone.gemfire.internal.cache.CacheServerLauncher !CLASS_ARGS!
-:done
-set scriptdir=
-set gf=
-set GEMFIRE_JARS=
-
-endlocal

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/scripts/cacheserver.sh
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/scripts/cacheserver.sh b/extensions/geode-modules-assembly/release/scripts/cacheserver.sh
deleted file mode 100755
index 28edc35..0000000
--- a/extensions/geode-modules-assembly/release/scripts/cacheserver.sh
+++ /dev/null
@@ -1,97 +0,0 @@
-#!/usr/bin/env bash
-# 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.
-#
-
-# Set GEMFIRE to the product toplevel directory
-GEMFIRE=`dirname $0`
-OLDPWD=$PWD
-cd $GEMFIRE
-GEMFIRE=`dirname $PWD`
-cd $OLDPWD
-
-TOMCAT_VER=`cat "${GEMFIRE}/conf/tomcat.version"`
-
-if [ -f $GEMFIRE/bin/modules.env ]; then
-  # Pull in TOMCAT_MAJOR_VER
-  . $GEMFIRE/bin/modules.env
-fi
-
-# Pull out the installation directory arguments passed in
-ARGS=( "$@" )
-ARGS_LENGTH=${#ARGS[@]}
-CLASS_ARGS=()
-for (( i=0; i<$ARGS_LENGTH; i++ ));
-do
-	if [ "${ARGS[$i]}" == "-d" ]; then
-		i=$(($i+1))
-	else 
-		CLASS_ARGS="${CLASS_ARGS} ${ARGS[$i]}"
-	fi
-done
-# End pulling out arguments
-
-# See if the user specified the tomcat installation directory location
-while [ $# -gt 0 ]; do
-  case $1 in
-    -d )
-      TC_INSTALL_DIR="$2"
-      break
-      ;;
-  esac
-  shift
-done
-
-
-if [[ -n $TC_INSTALL_DIR && -d $TC_INSTALL_DIR ]]; then
-  TOMCAT_DIR="$TC_INSTALL_DIR/tomcat-${TOMCAT_VER}"
-else
-  TOMCAT_DIR=`ls -d "${GEMFIRE}"/../tomcat-${TOMCAT_MAJOR_VER}* 2> /dev/null`
-fi
-
-if [[ -z "$TOMCAT_DIR" || ! -f "$TOMCAT_DIR/lib/catalina.jar" ]]; then
-  echo "ERROR: Could not determine TOMCAT library location."
-  echo "       Use the -d <tc Server installation directory> option."
-  echo "       Example: ./cacheserver.sh start -d /opt/pivotal/tcserver/pivotal-tc-server-standard"
-  exit 1
-fi
-
-if [ "x$WINDIR" != "x" ]; then
-  echo "ERROR: The variable WINDIR is set indicating this script is running in a Windows OS, please use the .bat file version instead."
-  exit 1
-fi
-
-GEMFIRE_DEP_JAR=$GEMFIRE/lib/geode-dependencies.jar
-if [ ! -f "$GEMFIRE_DEP_JAR" ]; then
-  echo "ERROR: Could not determine GEMFIRE location."
-  exit 1
-fi
-
-MOD_JAR=`ls $GEMFIRE/lib/geode-modules-?.*.jar` 2>/dev/null
-if [ -z "$MOD_JAR" ]; then
-  MOD_JAR=$GEMFIRE/lib/geode-modules.jar
-fi
-
-# Add Tomcat classes
-GEMFIRE_JARS=$GEMFIRE_DEP_JAR:$MOD_JAR:$TOMCAT_DIR/lib/servlet-api.jar:$TOMCAT_DIR/lib/catalina.jar:$TOMCAT_DIR/lib/tomcat-util.jar:$TOMCAT_DIR/bin/tomcat-juli.jar
-
-# Add configuration
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/conf
-
-if [ "x$CLASSPATH" != "x" ]; then
-  GEMFIRE_JARS=$GEMFIRE_JARS:$CLASSPATH
-fi
-
-${GF_JAVA:-java} ${JAVA_ARGS} -classpath ${GEMFIRE_JARS} com.gemstone.gemfire.internal.cache.CacheServerLauncher ${CLASS_ARGS}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/scripts/gemfire.bat
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/scripts/gemfire.bat b/extensions/geode-modules-assembly/release/scripts/gemfire.bat
deleted file mode 100755
index 6102b88..0000000
--- a/extensions/geode-modules-assembly/release/scripts/gemfire.bat
+++ /dev/null
@@ -1,41 +0,0 @@
-@echo off
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem      http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-@setlocal enableextensions
-@set scriptdir=%~dp0
-@set gf=%scriptdir:\bin\=%
-@if exist "%gf%\lib\gemfire.jar" @goto gfok
-@echo Could not determine GEMFIRE location
-@verify other 2>nul
-@goto done
-:gfok
-
-@set GEMFIRE_JARS=%gf%/lib/gemfire.jar;%gf%/lib/antlr.jar;%gf%/lib/mail.jar
-@if defined CLASSPATH set GEMFIRE_JARS=%GEMFIRE_JARS%;%CLASSPATH%
-
-@if not defined GF_JAVA (
-@REM %GF_JAVA% is not defined, assume it is on the PATH
-@set GF_JAVA=java
-)
-
-@"%GF_JAVA%" %JAVA_ARGS% -classpath "%GEMFIRE_JARS%" com.gemstone.gemfire.internal.SystemAdmin %*
-:done
-@set scriptdir=
-@set gf=
-@set GEMFIRE_JARS=
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/scripts/gemfire.sh
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/scripts/gemfire.sh b/extensions/geode-modules-assembly/release/scripts/gemfire.sh
deleted file mode 100755
index 12956cc..0000000
--- a/extensions/geode-modules-assembly/release/scripts/gemfire.sh
+++ /dev/null
@@ -1,58 +0,0 @@
-#!/bin/bash
-# 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.
-#
-
-# Set GEMFIRE to the product toplevel directory
-GEMFIRE=`dirname $0`
-OLDPWD=$PWD
-cd $GEMFIRE
-GEMFIRE=`dirname $PWD`
-cd $OLDPWD
-
-if [ "x$WINDIR" != "x" ]; then
-  echo "ERROR: The variable WINDIR is set indicating this script is running in a Windows OS, please use the .bat file version instead."
-  exit 1
-fi
-
-GEMFIRE_DEP_JAR=$GEMFIRE/lib/geode-dependencies.jar
-if [ ! -f "$GEMFIRE_DEP_JAR" ]; then
-  echo "ERROR: Could not determine GEMFIRE location."
-  exit 1
-fi
-
-GEMFIRE_JARS=$GEMFIRE_DEP_JAR
-
-if [ "x$CLASSPATH" != "x" ]; then
-  GEMFIRE_JARS=$GEMFIRE_JARS:$CLASSPATH
-fi
-
-# Command line args that start with -J will be passed to the java vm in JARGS.
-# See java --help for a listing of valid vm args.
-# Example: -J-Xmx1g sets the max heap size to 1 gigabyte.
-
-JARGS=
-GEMFIRE_ARGS=
-for i in "$@"
-do
-  if [ "-J" == "${i:0:2}" ]
-  then
-    JARGS="${JARGS} \"${i#-J}\""
-  else
-    GEMFIRE_ARGS="${GEMFIRE_ARGS} \"${i}\""
-  fi
-done
-
-eval ${GF_JAVA:-java} ${JAVA_ARGS} ${JARGS} -classpath ${GEMFIRE_JARS} com.gemstone.gemfire.internal.SystemAdmin ${GEMFIRE_ARGS}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/scripts/setenv.properties
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/scripts/setenv.properties b/extensions/geode-modules-assembly/release/scripts/setenv.properties
index fc8918b..a32aac4 100644
--- a/extensions/geode-modules-assembly/release/scripts/setenv.properties
+++ b/extensions/geode-modules-assembly/release/scripts/setenv.properties
@@ -3,4 +3,5 @@ java.opt.2=-Xmx${maximum.vm.heap.size.mb:512}M
 java.opt.hotspot.1=-XX:+UseParNewGC
 java.opt.hotspot.2=-XX:+UseConcMarkSweepGC
 java.opt.hotspot.3=-XX:CMSInitiatingOccupancyFraction=${cms.initiating.heap.percentage:50}
+java.opt.hotspot.4=-Dgemfire.disableShutdownHook=true
 java.opt.j9.1=-Xgcpolicy:gencon
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/session/bin/cacheserver.bat
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/session/bin/cacheserver.bat b/extensions/geode-modules-assembly/release/session/bin/cacheserver.bat
deleted file mode 100755
index 2935cde..0000000
--- a/extensions/geode-modules-assembly/release/session/bin/cacheserver.bat
+++ /dev/null
@@ -1,56 +0,0 @@
-@echo off
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem      http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-setlocal enableextensions
-set scriptdir=%~dp0
-set gf=%scriptdir:\bin\=%
-
-set GEMFIRE_JARS=%gf%\lib\gemfire.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\antlr.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\spring-core-3.1.1.RELEASE.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\spring-shell-1.0.0.RC1.jar
-if exist "%GEMFIRE_JARS%" goto gfok
-echo Could not determine GEMFIRE location
-verify other 2>nul
-goto done
-:gfok
-
-REM Initialize classpath
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\geode-modules-@GEMFIRE_MODULES_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\geode-modules-session-@GEMFIRE_MODULES_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\geode-modules-session-external-@GEMFIRE_MODULES_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\servlet-api-@SERVLET_API_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\slf4j-api-@SLF4J_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\slf4j-jdk14-@SLF4J_VERSION@.jar
-
-REM Add conf directory
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\conf
-
-if defined CLASSPATH set GEMFIRE_JARS=%GEMFIRE_JARS%;%CLASSPATH%
-
-if not defined GF_JAVA (
-REM %GF_JAVA% is not defined, assume it is on the PATH
-set GF_JAVA=java
-)
-
-"%GF_JAVA%" %JAVA_ARGS% -classpath "%GEMFIRE_JARS%" com.gemstone.gemfire.internal.cache.CacheServerLauncher %*
-:done
-set scriptdir=
-set gf=
-set GEMFIRE_JARS=
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/session/bin/cacheserver.sh
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/session/bin/cacheserver.sh b/extensions/geode-modules-assembly/release/session/bin/cacheserver.sh
deleted file mode 100755
index 2c12b57..0000000
--- a/extensions/geode-modules-assembly/release/session/bin/cacheserver.sh
+++ /dev/null
@@ -1,57 +0,0 @@
-#!/bin/bash
-# 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.
-#
-
-# Set GEMFIRE to the product toplevel directory
-GEMFIRE=`dirname $0`
-OLDPWD=$PWD
-cd $GEMFIRE
-GEMFIRE=`dirname $PWD`
-cd $OLDPWD
-
-if [ "x$WINDIR" != "x" ]; then
-  echo "ERROR: The variable WINDIR is set indicating this script is running in a Windows OS, please use the .bat file version instead."
-  exit 1
-fi
-
-GEMFIRE_JARS=$GEMFIRE/lib/gemfire.jar
-
-if [ ! -f "${GEMFIRE_JARS}" ]; then
-  echo "ERROR: Could not determine GEMFIRE location."
-  exit 1
-fi
-
-LOG4J_API=$( ls $GEMFIRE/lib/log4j-api*jar )
-LOG4J_CORE=$( ls $GEMFIRE/lib/log4j-core*jar )
-
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/antlr.jar:$LOG4J_API:$LOG4J_CORE
-
-# Initialize classpath
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/geode-modules-@GEMFIRE_MODULES_VERSION@.jar \
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/geode-modules-session-@GEMFIRE_MODULES_VERSION@.jar \
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/servlet-api-@SERVLET_API_VERSION@.jar \
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/geode-modules-session-external-@GEMFIRE_MODULES_VERSION@.jar \
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/slf4j-api-@SLF4J_VERSION@.jar \
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/slf4j-jdk14-@SLF4J_VERSION@.jar
-
-# Add configuration
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/conf
-
-if [ "x$CLASSPATH" != "x" ]; then
-  GEMFIRE_JARS=$GEMFIRE_JARS:$CLASSPATH
-fi
-
-${GF_JAVA:-java} ${JAVA_ARGS} -classpath ${GEMFIRE_JARS} com.gemstone.gemfire.internal.cache.CacheServerLauncher "$@"

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/session/bin/gemfire.bat
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/session/bin/gemfire.bat b/extensions/geode-modules-assembly/release/session/bin/gemfire.bat
deleted file mode 100755
index a66964c..0000000
--- a/extensions/geode-modules-assembly/release/session/bin/gemfire.bat
+++ /dev/null
@@ -1,53 +0,0 @@
-@echo off
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements.  See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License.  You may obtain a copy of the License at
-rem
-rem      http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-setlocal enableextensions
-set scriptdir=%~dp0
-set gf=%scriptdir:\bin\=%
-
-set GEMFIRE_JARS=%gf%\lib\gemfire.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\antlr.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\spring-core-3.1.1.RELEASE.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\spring-shell-1.0.0.RC1.jar
-if exist "%GEMFIRE_JARS%" goto gfok
-echo Could not determine GEMFIRE location
-verify other 2>nul
-goto done
-:gfok
-
-REM Initialize classpath
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\geode-modules-@GEMFIRE_MODULES_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\geode-modules-session-@GEMFIRE_MODULES_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\geode-modules-session-external-@GEMFIRE_MODULES_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\servlet-api-@SERVLET_API_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\slf4j-api-@SLF4J_VERSION@.jar
-set GEMFIRE_JARS=%GEMFIRE_JARS%;%gf%\lib\slf4j-jdk14-@SLF4J_VERSION@.jar
-
-if defined CLASSPATH set GEMFIRE_JARS=%GEMFIRE_JARS%;%CLASSPATH%
-
-if not defined GF_JAVA (
-REM %GF_JAVA% is not defined, assume it is on the PATH
-set GF_JAVA=java
-)
-
-"%GF_JAVA%" %JAVA_ARGS% -classpath "%GEMFIRE_JARS%" com.gemstone.gemfire.internal.SystemAdmin %*
-:done
-set scriptdir=
-set gf=
-set GEMFIRE_JARS=
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/session/bin/gemfire.sh
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/session/bin/gemfire.sh b/extensions/geode-modules-assembly/release/session/bin/gemfire.sh
deleted file mode 100755
index ac11a4d..0000000
--- a/extensions/geode-modules-assembly/release/session/bin/gemfire.sh
+++ /dev/null
@@ -1,69 +0,0 @@
-#!/bin/bash
-# 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.
-#
-
-# Set GEMFIRE to the product toplevel directory
-GEMFIRE=`dirname $0`
-OLDPWD=$PWD
-cd $GEMFIRE
-GEMFIRE=`dirname $PWD`
-cd $OLDPWD
-
-if [ "x$WINDIR" != "x" ]; then
-  echo "ERROR: The variable WINDIR is set indicating this script is running in a Windows OS, please use the .bat file version instead."
-  exit 1
-fi
-
-GEMFIRE_JARS=$GEMFIRE/lib/gemfire.jar
-if [ ! -f "${GEMFIRE_JARS}" ]; then
-  echo "ERROR: Could not determine GEMFIRE location."
-  exit 1
-fi
-
-LOG4J_API=$( ls $GEMFIRE/lib/log4j-api*jar )
-LOG4J_CORE=$( ls $GEMFIRE/lib/log4j-core*jar )
-
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/antlr.jar:$LOG4J_API:$LOG4J_CORE
-
-# Initialize classpath
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/geode-modules-@GEMFIRE_MODULES_VERSION@.jar
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/geode-modules-session-@GEMFIRE_MODULES_VERSION@.jar
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/geode-modules-session-external-@GEMFIRE_MODULES_VERSION@.jar
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/servlet-api-@SERVLET_API_VERSION@.jar
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/slf4j-api-@SLF4J_VERSION@.jar
-GEMFIRE_JARS=$GEMFIRE_JARS:$GEMFIRE/lib/slf4j-jdk14-@SLF4J_VERSION@.jar
-
-if [ "x$CLASSPATH" != "x" ]; then
-  GEMFIRE_JARS=$GEMFIRE_JARS:$CLASSPATH
-fi
-
-# Command line args that start with -J will be passed to the java vm in JARGS.
-# See java --help for a listing of valid vm args.
-# Example: -J-Xmx1g sets the max heap size to 1 gigabyte.
-
-JARGS=
-GEMFIRE_ARGS=
-for i in "$@"
-do
-  if [ "-J" == "${i:0:2}" ]
-  then
-    JARGS="${JARGS} \"${i#-J}\""
-  else
-    GEMFIRE_ARGS="${GEMFIRE_ARGS} \"${i}\""
-  fi
-done
-
-eval ${GF_JAVA:-java} ${JAVA_ARGS} ${JARGS} -classpath ${GEMFIRE_JARS} com.gemstone.gemfire.internal.SystemAdmin ${GEMFIRE_ARGS}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/session/bin/modify_war
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/session/bin/modify_war b/extensions/geode-modules-assembly/release/session/bin/modify_war
index 9c87104..7b5db21 100755
--- a/extensions/geode-modules-assembly/release/session/bin/modify_war
+++ b/extensions/geode-modules-assembly/release/session/bin/modify_war
@@ -15,41 +15,6 @@
 # limitations under the License.
 #
 
-X=`dirname $0`
-EXE_DIR=`cd $X; pwd`
-LIB_DIR=`cd ${X}/../lib; pwd`
-
-SESSION_JAR="${LIB_DIR}/geode-modules-session-external-@GEMFIRE_MODULES_VERSION@.jar"
-
-declare -a OTHER_JARS
-OTHER_JARS=(${LIB_DIR}/gemfire.jar \
-    ${LIB_DIR}/antlr.jar
-    ${LIB_DIR}/geode-modules-@GEMFIRE_MODULES_VERSION@.jar \
-    ${LIB_DIR}/geode-modules-session-@GEMFIRE_MODULES_VERSION@.jar \
-    ${LIB_DIR}/slf4j-api-@SLF4J_VERSION@.jar)
-
-# This is required for GemFire 8.1
-LOG4J_API=$(ls ${LIB_DIR}/log4j-api*jar)
-LOG4J_CORE=$(ls ${LIB_DIR}/log4j-core*jar)
-if [ -f ${LOG4J_API} -a -f ${LOG4J_CORE} ]; then
-    OTHER_JARS[${#OTHER_JARS[*]}]=$LOG4J_API
-    OTHER_JARS[${#OTHER_JARS[*]}]=$LOG4J_CORE
-fi
-
-TMP_DIR="/tmp/modify_war.$$"
-
-ARG_INPUT_FILE=""
-ARG_WAR_LIB_DIR="WEB-INF/lib"
-ARG_EAR_LIB_DIR="lib"
-ARG_OUTPUT_FILE=""
-ARG_TEST_RUN=0
-ARG_CACHE_TYPE="peer-to-peer"
-ARG_PROCESS_LIBS=1
-declare -a ARG_GEMFIRE_PARAMETERS=""
-declare -a ARG_WAR_LIBS="$SESSION_JAR"
-declare -a ARG_EAR_LIBS=""
-declare -a ARG_JVM_OPTS=""
-
 function usage() {
     cat <<EOF
 USAGE: modify_war <args>
@@ -119,13 +84,21 @@ WHERE <args>:
                     necessary jars into the file. When this option is used,
                     additional jars will need to be made available to the
                     container:
-                        gemfire.jar
                         geode-modules.jar
-                        geode-modules-session.jar
+                        geode-modules-session-internal.jar
+                        geode-core.jar
+                        geode-json.jar
+                        antlr.jar
+                        log4j-core.jar
+                        log4j-api.jar
+                        log4j-jul.jar
+                        fastutil.jar
+                        javax.transactions-api.jar
+                        jgroups.jar
                         slf4j-api.jar
                         slf4j-jdk14.jar (not for WebLogic)
                         geode-modules-slf4j-weblogic.jar (WebLogic only)
-                    This option still modifes any web.xml files.
+                    This option still modifies any web.xml files.
                         
 EOF
 }
@@ -272,6 +245,48 @@ function process_input_file() {
     done
 }
 
+########  Mainline  #########
+
+X=`dirname $0`
+EXE_DIR=`cd $X; pwd`
+LIB_DIR=`cd ${X}/../lib; pwd`
+VERSION="@GEODE_VERSION@"
+
+if [ -z "$GEODE" ]; then
+    exiting "Please set the GEODE environment variable to the root of the Geode install location" 1
+fi
+
+SESSION_JAR="${LIB_DIR}/geode-modules-session-${VERSION}.jar"
+
+declare -a OTHER_JARS
+OTHER_JARS=(${GEODE}/lib/geode-core-${VERSION}.jar \
+    ${GEODE}/lib/geode-json-${VERSION}.jar \
+    ${GEODE}/lib/antlr-@ANTLR_VERSION@.jar \
+    ${GEODE}/lib/log4j-core-@LOG4J_VERSION@.jar \
+    ${GEODE}/lib/log4j-api-@LOG4J_VERSION@.jar \
+    ${GEODE}/lib/log4j-jul-@LOG4J_VERSION@.jar \
+    ${GEODE}/lib/fastutil-@FASTUTIL_VERSION@.jar \
+    ${GEODE}/lib/javax.transaction-api-@TX_VERSION@.jar \
+    ${GEODE}/lib/jgroups-@JGROUPS_VERSION@.jar
+    ${LIB_DIR}/geode-modules-${VERSION}.jar \
+    ${LIB_DIR}/geode-modules-session-internal-${VERSION}.jar \
+    ${LIB_DIR}/slf4j-api-@SLF4J_VERSION@.jar \
+    ${LIB_DIR}/slf4j-jdk14-@SLF4J_VERSION@.jar)
+
+TMP_DIR="/tmp/modify_war.$$"
+
+ARG_INPUT_FILE=""
+ARG_WAR_LIB_DIR="WEB-INF/lib"
+ARG_EAR_LIB_DIR="lib"
+ARG_OUTPUT_FILE=""
+ARG_TEST_RUN=0
+ARG_CACHE_TYPE=""
+ARG_PROCESS_LIBS=1
+declare -a ARG_GEMFIRE_PARAMETERS=""
+declare -a ARG_WAR_LIBS="$SESSION_JAR"
+declare -a ARG_EAR_LIBS=""
+declare -a ARG_JVM_OPTS=""
+
 trap exiting INT QUIT TERM
 
 mkdir $TMP_DIR

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/session/bin/setenv.properties
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/session/bin/setenv.properties b/extensions/geode-modules-assembly/release/session/bin/setenv.properties
deleted file mode 100644
index fc8918b..0000000
--- a/extensions/geode-modules-assembly/release/session/bin/setenv.properties
+++ /dev/null
@@ -1,6 +0,0 @@
-java.opt.1=-Xms${initial.vm.heap.size.mb:512}M
-java.opt.2=-Xmx${maximum.vm.heap.size.mb:512}M
-java.opt.hotspot.1=-XX:+UseParNewGC
-java.opt.hotspot.2=-XX:+UseConcMarkSweepGC
-java.opt.hotspot.3=-XX:CMSInitiatingOccupancyFraction=${cms.initiating.heap.percentage:50}
-java.opt.j9.1=-Xgcpolicy:gencon
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/context-fragment.xml
deleted file mode 100644
index 72faee2..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/context-fragment.xml
+++ /dev/null
@@ -1,15 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Context>
-
-    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
-
-    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
-      enableDebugListener="${enable.debug.listener:false}"
-      enableGatewayReplication="${enable.gateway.replication:false}"
-      enableLocalCache="${enable.local.cache:true}"
-      enableCommitValve="${enable.commit.valve:true}"
-      preferDeserializedForm="${prefer.deserialized.form:true}"
-      regionAttributesId="${region.attributes.id:PARTITION_REDUNDANT}"
-      regionName="${region.name:gemfire_modules_sessions}"/>
-
-</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/modules.env
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/modules.env b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/modules.env
deleted file mode 100644
index 1c14a6c..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-7/modules.env
+++ /dev/null
@@ -1 +0,0 @@
-TOMCAT_MAJOR_VER=7

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/context-fragment.xml
deleted file mode 100644
index 72faee2..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/context-fragment.xml
+++ /dev/null
@@ -1,15 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Context>
-
-    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
-
-    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
-      enableDebugListener="${enable.debug.listener:false}"
-      enableGatewayReplication="${enable.gateway.replication:false}"
-      enableLocalCache="${enable.local.cache:true}"
-      enableCommitValve="${enable.commit.valve:true}"
-      preferDeserializedForm="${prefer.deserialized.form:true}"
-      regionAttributesId="${region.attributes.id:PARTITION_REDUNDANT}"
-      regionName="${region.name:gemfire_modules_sessions}"/>
-
-</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/modules.env
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/modules.env b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/modules.env
deleted file mode 100644
index d220675..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs-tomcat-8/modules.env
+++ /dev/null
@@ -1 +0,0 @@
-TOMCAT_MAJOR_VER=8

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/configuration-prompts.properties
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/configuration-prompts.properties b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/configuration-prompts.properties
deleted file mode 100644
index 0ebae97..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/configuration-prompts.properties
+++ /dev/null
@@ -1,17 +0,0 @@
-cache.configuration.file=Please enter the name of the GemFire cache configuration file. Default '${default}':
-critical.heap.percentage=Please enter the percentage of heap at which updates to the cache are refused. 0.0 means disabled. Default '${default}':
-enable.debug.listener=Please specify whether to enable a GemFire listener that logs session create, update, destroy and expiration events. Default '${default}':
-enable.gateway.replication=Please specify whether session modifications should be replicated across the WAN. Default '${default}':
-enable.local.cache=Please specify whether to maintain a local GemFire cache. Default '${default}':
-enable.commit.valve=Please specify whether to commit sessions once per request. Default '${default}':
-prefer.deserialized.form=Please specify whether to prefer keeping attributes in deserialized form. Default '${default}':
-eviction.heap.percentage=Please enter the percentage of heap at which sessions will be evicted from the local cache. Default '${default}':
-log.file=Please enter the name of the file used to log GemFire messages. Default '${default}':
-region.attributes.id=Please enter the id of the attributes of the GemFire region used to cache sessions. Default '${default}':
-region.name=Please enter the name of the GemFire region used to cache sessions. Default '${default}':
-statistic.archive.file=Please enter the name of the file used to store GemFire statistics. Default '${default}':
-statistic.sampling.enabled=Please specify whether GemFire statistic sampling should be enabled. Default '${default}':
-initial.vm.heap.size.mb=Please specify the initial VM heap size in MB (-Xms). Default '${default}':
-maximum.vm.heap.size.mb=Please specify the maximum VM heap size in MB (-Xmx). Default '${default}':
-cms.initiating.heap.percentage=Please specify the percentage of VM heap utilization before a concurrent collection is initiated (--XX:CMSInitiatingOccupancyFraction [Hotspot-only]). Default '${default}':
-tomcat.version=Please enter the major version of Tomcat you will be using (6, 7 or 8). Default '${default}'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/context-fragment.xml
deleted file mode 100644
index fac3044..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/context-fragment.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Context>
-
-    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"
-      enableDebugListener="${enable.debug.listener:false}"
-      enableGatewayReplication="${enable.gateway.replication:false}"
-      enableLocalCache="${enable.local.cache:true}"
-      enableCommitValve="${enable.commit.valve:true}"
-      preferDeserializedForm="${prefer.deserialized.form:true}"
-      regionAttributesId="${region.attributes.id:PARTITION_REDUNDANT}"
-      regionName="${region.name:gemfire_modules_sessions}"/>
-
-</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/modules.env
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/modules.env b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/modules.env
deleted file mode 100644
index 9c4c00a..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/modules.env
+++ /dev/null
@@ -1 +0,0 @@
-TOMCAT_MAJOR_VER=6

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/server-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/server-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/server-fragment.xml
deleted file mode 100644
index c4adc66..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-cs/server-fragment.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version='1.0' encoding='utf-8'?>
-<Server>
-
-    <add:Listener className="com.gemstone.gemfire.modules.session.catalina.ClientServerCacheLifecycleListener"
-      cache-xml-file="${cache.configuration.file:cache-client.xml}"
-      criticalHeapPercentage="${critical.heap.percentage:0.0}"
-      evictionHeapPercentage="${eviction.heap.percentage:80.0}"
-      log-file="${log.file:gemfire_modules.log}"
-      statistic-archive-file="${statistic.archive.file:gemfire_modules.gfs}"
-      statistic-sampling-enabled="${statistic.sampling.enabled:false}"/>
-
-</Server>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/context-fragment.xml
deleted file mode 100644
index 155aa2a..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/context-fragment.xml
+++ /dev/null
@@ -1,15 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Context>
-
-    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
-
-    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
-      enableDebugListener="${enable.debug.listener:false}"
-      enableGatewayReplication="${enable.gateway.replication:false}"
-      enableLocalCache="${enable.local.cache:false}"
-      enableCommitValve="${enable.commit.valve:true}"
-      preferDeserializedForm="${prefer.deserialized.form:true}"
-      regionAttributesId="${region.attributes.id:REPLICATE}"
-      regionName="${region.name:gemfire_modules_sessions}"/>
-
-</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/modules.env
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/modules.env b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/modules.env
deleted file mode 100644
index 1c14a6c..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-7/modules.env
+++ /dev/null
@@ -1 +0,0 @@
-TOMCAT_MAJOR_VER=7

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/context-fragment.xml
deleted file mode 100644
index 155aa2a..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/context-fragment.xml
+++ /dev/null
@@ -1,15 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Context>
-
-    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
-
-    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
-      enableDebugListener="${enable.debug.listener:false}"
-      enableGatewayReplication="${enable.gateway.replication:false}"
-      enableLocalCache="${enable.local.cache:false}"
-      enableCommitValve="${enable.commit.valve:true}"
-      preferDeserializedForm="${prefer.deserialized.form:true}"
-      regionAttributesId="${region.attributes.id:REPLICATE}"
-      regionName="${region.name:gemfire_modules_sessions}"/>
-
-</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/modules.env
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/modules.env b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/modules.env
deleted file mode 100644
index d220675..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p-tomcat-8/modules.env
+++ /dev/null
@@ -1 +0,0 @@
-TOMCAT_MAJOR_VER=8

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/configuration-prompts.properties
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/configuration-prompts.properties b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/configuration-prompts.properties
deleted file mode 100644
index fdead44..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/configuration-prompts.properties
+++ /dev/null
@@ -1,19 +0,0 @@
-cache.configuration.file=Please enter the name of the GemFire cache configuration file. Default '${default}':
-critical.heap.percentage=Please enter the percentage of heap at which updates to the cache are refused. 0.0 means disabled. Default '${default}':
-enable.debug.listener=Please specify whether to enable a GemFire listener that logs session create, update, destroy and expiration events. Default '${default}':
-enable.gateway.replication=Please specify whether session modifications should be replicated across the WAN. Default '${default}':
-enable.local.cache=Please specify whether to maintain a local GemFire cache. Default '${default}':
-enable.commit.valve=Please specify whether to commit sessions once per request. Default '${default}':
-prefer.deserialized.form=Please specify whether to prefer keeping attributes in deserialized form. Default '${default}':
-eviction.heap.percentage=Please enter the percentage of heap at which sessions will be evicted from the local cache. Default '${default}':
-locators=Please enter the list of locators used by GemFire members to discover each other. The format is a comma-separated list of host[port]. Default '${default}':
-log.file=Please enter the name of the file used to log GemFire messages. Default '${default}':
-rebalance=Please specify whether to rebalance the GemFire cache at startup. Default '${default}':
-region.attributes.id=Please enter the id of the attributes of the GemFire region used to cache sessions. Default '${default}':
-region.name=Please enter the name of the GemFire region used to cache sessions. Default '${default}':
-statistic.archive.file=Please enter the name of the file used to store GemFire statistics. Default '${default}':
-statistic.sampling.enabled=Please specify whether GemFire statistic sampling should be enabled. Default '${default}':
-initial.vm.heap.size.mb=Please specify the initial VM heap size in MB (-Xms). Default '${default}':
-maximum.vm.heap.size.mb=Please specify the maximum VM heap size in MB (-Xmx). Default '${default}':
-cms.initiating.heap.percentage=Please specify the percentage of VM heap utilization before a concurrent collection is initiated (--XX:CMSInitiatingOccupancyFraction [Hotspot-only]). Default '${default}':
-tomcat.version=Please enter the major version of Tomcat you will be using (6, 7 or 8). Default '${default}':

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/context-fragment.xml
deleted file mode 100644
index 2b3d25c..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/context-fragment.xml
+++ /dev/null
@@ -1,13 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Context>
-
-    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"
-      enableDebugListener="${enable.debug.listener:false}"
-      enableGatewayReplication="${enable.gateway.replication:false}"
-      enableLocalCache="${enable.local.cache:false}"
-      enableCommitValve="${enable.commit.valve:true}"
-      preferDeserializedForm="${prefer.deserialized.form:true}"
-      regionAttributesId="${region.attributes.id:REPLICATE}"
-      regionName="${region.name:gemfire_modules_sessions}"/>
-
-</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/modules.env
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/modules.env b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/modules.env
deleted file mode 100644
index 9c4c00a..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/modules.env
+++ /dev/null
@@ -1 +0,0 @@
-TOMCAT_MAJOR_VER=6

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/server-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/server-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/server-fragment.xml
deleted file mode 100644
index cf7dc0d..0000000
--- a/extensions/geode-modules-assembly/release/tcserver/gemfire-p2p/server-fragment.xml
+++ /dev/null
@@ -1,14 +0,0 @@
-<?xml version='1.0' encoding='utf-8'?>
-<Server>
-
-    <add:Listener className="com.gemstone.gemfire.modules.session.catalina.PeerToPeerCacheLifecycleListener"
-      cache-xml-file="${cache.configuration.file:cache-peer.xml}"
-      criticalHeapPercentage="${critical.heap.percentage:0.0}"
-      evictionHeapPercentage="${eviction.heap.percentage:80.0}"
-      locators="${locators:localhost[10334]}"
-      log-file="${log.file:gemfire_modules.log}"
-      rebalance="${rebalance:false}"
-      statistic-archive-file="${statistic.archive.file:gemfire_modules.gfs}"
-      statistic-sampling-enabled="${statistic.sampling.enabled:false}"/>
-
-</Server>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-7/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-7/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-7/context-fragment.xml
new file mode 100644
index 0000000..72faee2
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-7/context-fragment.xml
@@ -0,0 +1,15 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Context>
+
+    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
+
+    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
+      enableDebugListener="${enable.debug.listener:false}"
+      enableGatewayReplication="${enable.gateway.replication:false}"
+      enableLocalCache="${enable.local.cache:true}"
+      enableCommitValve="${enable.commit.valve:true}"
+      preferDeserializedForm="${prefer.deserialized.form:true}"
+      regionAttributesId="${region.attributes.id:PARTITION_REDUNDANT}"
+      regionName="${region.name:gemfire_modules_sessions}"/>
+
+</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-8/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-8/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-8/context-fragment.xml
new file mode 100644
index 0000000..72faee2
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-cs-tomcat-8/context-fragment.xml
@@ -0,0 +1,15 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Context>
+
+    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
+
+    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
+      enableDebugListener="${enable.debug.listener:false}"
+      enableGatewayReplication="${enable.gateway.replication:false}"
+      enableLocalCache="${enable.local.cache:true}"
+      enableCommitValve="${enable.commit.valve:true}"
+      preferDeserializedForm="${prefer.deserialized.form:true}"
+      regionAttributesId="${region.attributes.id:PARTITION_REDUNDANT}"
+      regionName="${region.name:gemfire_modules_sessions}"/>
+
+</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-cs/configuration-prompts.properties
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-cs/configuration-prompts.properties b/extensions/geode-modules-assembly/release/tcserver/geode-cs/configuration-prompts.properties
new file mode 100644
index 0000000..391b86f
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-cs/configuration-prompts.properties
@@ -0,0 +1,17 @@
+cache.configuration.file=Please enter the name of the Geode cache configuration file. Default '${default}':
+critical.heap.percentage=Please enter the percentage of heap at which updates to the cache are refused. 0.0 means disabled. Default '${default}':
+enable.debug.listener=Please specify whether to enable a Geode listener that logs session create, update, destroy and expiration events. Default '${default}':
+enable.gateway.replication=Please specify whether session modifications should be replicated across the WAN. Default '${default}':
+enable.local.cache=Please specify whether to maintain a local Geode cache. Default '${default}':
+enable.commit.valve=Please specify whether to commit sessions once per request. Default '${default}':
+prefer.deserialized.form=Please specify whether to prefer keeping attributes in deserialized form. Default '${default}':
+eviction.heap.percentage=Please enter the percentage of heap at which sessions will be evicted from the local cache. Default '${default}':
+log.file=Please enter the name of the file used to log Geode messages. Default '${default}':
+region.attributes.id=Please enter the id of the attributes of the Geode region used to cache sessions. Default '${default}':
+region.name=Please enter the name of the Geode region used to cache sessions. Default '${default}':
+statistic.archive.file=Please enter the name of the file used to store Geode statistics. Default '${default}':
+statistic.sampling.enabled=Please specify whether Geode statistic sampling should be enabled. Default '${default}':
+initial.vm.heap.size.mb=Please specify the initial VM heap size in MB (-Xms). Default '${default}':
+maximum.vm.heap.size.mb=Please specify the maximum VM heap size in MB (-Xmx). Default '${default}':
+cms.initiating.heap.percentage=Please specify the percentage of VM heap utilization before a concurrent collection is initiated (--XX:CMSInitiatingOccupancyFraction [Hotspot-only]). Default '${default}':
+tomcat.version=Please enter the major version of Tomcat you will be using (6, 7 or 8). Default '${default}'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-cs/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-cs/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-cs/context-fragment.xml
new file mode 100644
index 0000000..fac3044
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-cs/context-fragment.xml
@@ -0,0 +1,13 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Context>
+
+    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"
+      enableDebugListener="${enable.debug.listener:false}"
+      enableGatewayReplication="${enable.gateway.replication:false}"
+      enableLocalCache="${enable.local.cache:true}"
+      enableCommitValve="${enable.commit.valve:true}"
+      preferDeserializedForm="${prefer.deserialized.form:true}"
+      regionAttributesId="${region.attributes.id:PARTITION_REDUNDANT}"
+      regionName="${region.name:gemfire_modules_sessions}"/>
+
+</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-cs/server-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-cs/server-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-cs/server-fragment.xml
new file mode 100644
index 0000000..c4adc66
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-cs/server-fragment.xml
@@ -0,0 +1,12 @@
+<?xml version='1.0' encoding='utf-8'?>
+<Server>
+
+    <add:Listener className="com.gemstone.gemfire.modules.session.catalina.ClientServerCacheLifecycleListener"
+      cache-xml-file="${cache.configuration.file:cache-client.xml}"
+      criticalHeapPercentage="${critical.heap.percentage:0.0}"
+      evictionHeapPercentage="${eviction.heap.percentage:80.0}"
+      log-file="${log.file:gemfire_modules.log}"
+      statistic-archive-file="${statistic.archive.file:gemfire_modules.gfs}"
+      statistic-sampling-enabled="${statistic.sampling.enabled:false}"/>
+
+</Server>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-7/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-7/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-7/context-fragment.xml
new file mode 100644
index 0000000..155aa2a
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-7/context-fragment.xml
@@ -0,0 +1,15 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Context>
+
+    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
+
+    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
+      enableDebugListener="${enable.debug.listener:false}"
+      enableGatewayReplication="${enable.gateway.replication:false}"
+      enableLocalCache="${enable.local.cache:false}"
+      enableCommitValve="${enable.commit.valve:true}"
+      preferDeserializedForm="${prefer.deserialized.form:true}"
+      regionAttributesId="${region.attributes.id:REPLICATE}"
+      regionName="${region.name:gemfire_modules_sessions}"/>
+
+</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-8/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-8/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-8/context-fragment.xml
new file mode 100644
index 0000000..155aa2a
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-p2p-tomcat-8/context-fragment.xml
@@ -0,0 +1,15 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Context>
+
+    <remove:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"/>
+
+    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat7DeltaSessionManager"
+      enableDebugListener="${enable.debug.listener:false}"
+      enableGatewayReplication="${enable.gateway.replication:false}"
+      enableLocalCache="${enable.local.cache:false}"
+      enableCommitValve="${enable.commit.valve:true}"
+      preferDeserializedForm="${prefer.deserialized.form:true}"
+      regionAttributesId="${region.attributes.id:REPLICATE}"
+      regionName="${region.name:gemfire_modules_sessions}"/>
+
+</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-p2p/configuration-prompts.properties
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-p2p/configuration-prompts.properties b/extensions/geode-modules-assembly/release/tcserver/geode-p2p/configuration-prompts.properties
new file mode 100644
index 0000000..e739513
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-p2p/configuration-prompts.properties
@@ -0,0 +1,19 @@
+cache.configuration.file=Please enter the name of the Geode cache configuration file. Default '${default}':
+critical.heap.percentage=Please enter the percentage of heap at which updates to the cache are refused. 0.0 means disabled. Default '${default}':
+enable.debug.listener=Please specify whether to enable a Geode listener that logs session create, update, destroy and expiration events. Default '${default}':
+enable.gateway.replication=Please specify whether session modifications should be replicated across the WAN. Default '${default}':
+enable.local.cache=Please specify whether to maintain a local Geode cache. Default '${default}':
+enable.commit.valve=Please specify whether to commit sessions once per request. Default '${default}':
+prefer.deserialized.form=Please specify whether to prefer keeping attributes in deserialized form. Default '${default}':
+eviction.heap.percentage=Please enter the percentage of heap at which sessions will be evicted from the local cache. Default '${default}':
+locators=Please enter the list of locators used by Geode members to discover each other. The format is a comma-separated list of host[port]. Default '${default}':
+log.file=Please enter the name of the file used to log Geode messages. Default '${default}':
+rebalance=Please specify whether to rebalance the Geode cache at startup. Default '${default}':
+region.attributes.id=Please enter the id of the attributes of the Geode region used to cache sessions. Default '${default}':
+region.name=Please enter the name of the Geode region used to cache sessions. Default '${default}':
+statistic.archive.file=Please enter the name of the file used to store Geode statistics. Default '${default}':
+statistic.sampling.enabled=Please specify whether Geode statistic sampling should be enabled. Default '${default}':
+initial.vm.heap.size.mb=Please specify the initial VM heap size in MB (-Xms). Default '${default}':
+maximum.vm.heap.size.mb=Please specify the maximum VM heap size in MB (-Xmx). Default '${default}':
+cms.initiating.heap.percentage=Please specify the percentage of VM heap utilization before a concurrent collection is initiated (--XX:CMSInitiatingOccupancyFraction [Hotspot-only]). Default '${default}':
+tomcat.version=Please enter the major version of Tomcat you will be using (6, 7 or 8). Default '${default}':

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-p2p/context-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-p2p/context-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-p2p/context-fragment.xml
new file mode 100644
index 0000000..2b3d25c
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-p2p/context-fragment.xml
@@ -0,0 +1,13 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Context>
+
+    <add:Manager className="com.gemstone.gemfire.modules.session.catalina.Tomcat6DeltaSessionManager"
+      enableDebugListener="${enable.debug.listener:false}"
+      enableGatewayReplication="${enable.gateway.replication:false}"
+      enableLocalCache="${enable.local.cache:false}"
+      enableCommitValve="${enable.commit.valve:true}"
+      preferDeserializedForm="${prefer.deserialized.form:true}"
+      regionAttributesId="${region.attributes.id:REPLICATE}"
+      regionName="${region.name:gemfire_modules_sessions}"/>
+
+</Context>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-assembly/release/tcserver/geode-p2p/server-fragment.xml
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-assembly/release/tcserver/geode-p2p/server-fragment.xml b/extensions/geode-modules-assembly/release/tcserver/geode-p2p/server-fragment.xml
new file mode 100644
index 0000000..cf7dc0d
--- /dev/null
+++ b/extensions/geode-modules-assembly/release/tcserver/geode-p2p/server-fragment.xml
@@ -0,0 +1,14 @@
+<?xml version='1.0' encoding='utf-8'?>
+<Server>
+
+    <add:Listener className="com.gemstone.gemfire.modules.session.catalina.PeerToPeerCacheLifecycleListener"
+      cache-xml-file="${cache.configuration.file:cache-peer.xml}"
+      criticalHeapPercentage="${critical.heap.percentage:0.0}"
+      evictionHeapPercentage="${eviction.heap.percentage:80.0}"
+      locators="${locators:localhost[10334]}"
+      log-file="${log.file:gemfire_modules.log}"
+      rebalance="${rebalance:false}"
+      statistic-archive-file="${statistic.archive.file:gemfire_modules.gfs}"
+      statistic-sampling-enabled="${statistic.sampling.enabled:false}"/>
+
+</Server>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules-session/build.gradle
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/build.gradle b/extensions/geode-modules-session/build.gradle
index e301a6a..765bdd9 100644
--- a/extensions/geode-modules-session/build.gradle
+++ b/extensions/geode-modules-session/build.gradle
@@ -45,8 +45,7 @@ task internalJar(type: Jar) {
   from sourceSets.main.output
   include '**/internal/**/*'
   includeEmptyDirs = false
-  baseName = 'geode-modules-session'
-  classifier = 'internal'
+  baseName = 'geode-modules-session-internal'
 }
 
 artifacts {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/Banner.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/Banner.java b/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/Banner.java
index dbe8e05..413da07 100644
--- a/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/Banner.java
+++ b/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/Banner.java
@@ -26,19 +26,6 @@ import java.util.Properties;
 
 public class Banner {
 
-  private static String VERSION = "unknown";
-
-  private static Properties props = new Properties();
-
-  static {
-    InputStream is = Banner.class.getResourceAsStream("/modules-version.properties");
-    try {
-      props.load(is);
-      VERSION = props.getProperty("version");
-    } catch (IOException e) {
-    }
-  }
-
   public static String getString() {
     StringWriter sw = new StringWriter();
     PrintWriter pw = new PrintWriter(sw);
@@ -48,12 +35,7 @@ public class Banner {
   }
 
   private static void print(PrintWriter pw) {
-    pw.println("GemFire Modules");
-    pw.print("Modules version: ");
-    pw.println(VERSION);
+    pw.println("Geode Modules");
     GemFireVersion.print(pw);
   }
-
-  private Banner() {
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/geode-core/build.gradle
----------------------------------------------------------------------
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 8cd5bf3..4877e03 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -32,7 +32,7 @@ dependencies {
   // External 
   provided files("${System.getProperty('java.home')}/../lib/tools.jar")
   compile 'com.github.stephenc.findbugs:findbugs-annotations:' + project.'stephenc-findbugs.version'
-  compile 'org.jgroups:jgroups:3.6.8.Final'
+  compile 'org.jgroups:jgroups:' + project.'jgroups.version'
   compile 'antlr:antlr:' + project.'antlr.version'
   compile 'com.fasterxml.jackson.core:jackson-databind:' + project.'jackson.version'
   compile 'commons-io:commons-io:' + project.'commons-io.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d216e553/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 640b175..b6687e5 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -63,6 +63,7 @@ javax.servlet-api.version = 3.1.0
 javax.transaction-api.version = 1.2
 jedis.version = 2.7.2
 jetty.version = 9.3.6.v20151106
+jgroups.version = 3.6.8.Final
 jline.version = 2.12
 jmock.version = 2.8.2
 jna.version = 4.0.0